You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by Apache Wiki <wi...@apache.org> on 2010/02/02 00:09:38 UTC

[Pig Wiki] Update of "PigLogicalPlanOptimizerRewrite" by AlanGates

Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Pig Wiki" for change notification.

The "PigLogicalPlanOptimizerRewrite" page has been changed by AlanGates.
http://wiki.apache.org/pig/PigLogicalPlanOptimizerRewrite?action=diff&rev1=1&rev2=2

--------------------------------------------------

  == Problem Statement ==
  The current implementation of the logical plan and the logical optimizer in Pig has proven to not be easily extensible.  Developer feedback has indicated that adding new
  rules to the optimizer is quite burdensome.  In addition, the logical plan has been an area of numerous bugs, many of which have been difficult to fix.  Developers also feel
- that the logical plan is difficult to understand and maintain.  The root cause for these issues is that a number of design decisions that were made as part of the 0.2 rewrite of the front end have now proven to be
+ that the logical plan is difficult to understand and maintain.  The root cause for these issues is that a number of design decisions that were made as part of the 0.2 rewrite
+ of the front end have now proven to be
  sub-optimal.  The heart of this proposal is to revisit a number of those proposals and rebuild the logical plan with a simpler design that will make it much easier to
  maintain the logical plan as well as extend the logical optimizer.
  
  === Issues that Need to be Addressed in this Rework ===
  '''One:'''  !OperatorPlan has far too many operations.  It has 29 public methods.  This needs to be paired down to a minimal set of operators that are well defined.
  
- '''Two:''' Currently, relational operators (Join, Sort, etc.) and expression operators (add, equals, etc.) are both !LogicalOperators.  Operators such as Cogroup that contain expressions have !OperatorPlans that contain these expressions.  This was done for two reasons:
+ '''Two:''' Currently, relational operators (Join, Sort, etc.) and expression operators (add, equals, etc.) are both !LogicalOperators.  Operators such as Cogroup that contain
+ expressions have !OperatorPlans that contain these expressions.  This was done for two reasons:
   1. To make it easier for visitors to visit both types of operators (that is, visitors didn't have to have separate logic to handle expressions).
   1. To better handle the ambiguous nature of inner plans in Foreach.
- However, it has led to visitors and graphs that are hard to understand.  Both of the above concerns can be handled while breaking this binding so that relational and expression operators are seaprate types.
+ However, it has led to visitors and graphs that are hard to understand.  Both of the above concerns can be handled while breaking this binding so that relational and expression
+ operators are separate types.
  
  '''Three:'''  Related to the issue of relational and expression operators sharing a type is that inner plans have connections to outer plans.  Take for example a script like
  
@@ -24, +27 @@

  D = filter C by A.x > 0;
  }}}
  In this case the cogroup will have two inner plans, one of which will be a project of A.x and the other a project B.u.  The !LOProject objects representing these projections
- will hold actual references to the !LOLoad operators for A and B.  This makes disconecting and rearranging nodes in the plan much more difficult.  Consider if the optimizer wants to
+ will hold actual references to the !LOLoad operators for A and B.  This makes disconnecting and rearranging nodes in the plan much more difficult.  Consider if the optimizer wants to
  move the filter in D above C.  Now it has to not only change connections in the outer plan between load, cogroup, and filter; it also has to change connections in the
  first inner plan of C, because this now needs to point to the !LOFilter for D rather than the !LOLoad for A.
  
  '''Four:'''  The work done on Operator and !OperatorPlan to support the original rules for the optimizer had two main problems:
-    1.. The set of primitives chosen were not the correct ones.
+  1. The set of primitives chosen were not the correct ones.
-    1.. The operations chosen were put on the generic super classes (Operator) rather than further down on the specific classes that would know how to implement them.
+  1. The operations chosen were put on the generic super classes (Operator) rather than further down on the specific classes that would know how to implement them.
  
  '''Five:'''  At a number of points efforts were made to keep the logical plan close to the physical plan.  For example, !LOProject represents all of the same operations that
- !POProject does.  While this is convenient in translation, it is not convenient when trying to optimize the plan.  The !LogicalPlan needs to focus on reprenting the logic of
+ !POProject does.  While this is convenient in translation, it is not convenient when trying to optimize the plan.  The !LogicalPlan needs to focus on representing the logic of
  the script in a way that is easy for semantic checkers (such as !TypeChecker) and the optimizer to work with.
  
  '''Six:'''  The rule of one operation per operator was violated.  !LOProject handles three separate roles (converting from a relational to an expression operator, actually
@@ -45, +48 @@

  == Proposed Methodology ==
  Fixing these issues will require extensive changes, including a complete rewrite of Operator, !OperatorPlan, !PlanVisitor, !LogicalOperator, !LogicalPlan,
  !LogicalPlanVisitor, every current subclass of !LogicalOperator, and all existing optimizer rules.  It will also require extensive changes, though not complete rewrites, in
- existing subclasses of !LogicalTransformer.  To avoid destablizing the entire codebase during this operation, this will be done in a new set of packages as a totally separate
+ existing subclasses of !LogicalTransformer.  To avoid destabilizing the entire codebase during this operation, this will be done in a new set of packages as a totally separate
  set of classes.  Linkage code will be written to translate the current !LogicalPlan to the new experimental !LogicalPlan class.  A new !LogicalToPhysicalTranslator will also
  be written to translate this new !LogicalPlan to a !PhysicalPlan.  This code path will only be taken if some type of command line switch or property is set, thus insulating
  current developers and users from this work.
@@ -56, +59 @@

  == The Actual Proposal ==
  
  === Changes to Plans ===
- In general, the generic plan classes will be changing in a couple of important ways:
+ In general, the top level plan classes will be changing in a couple of important ways:
  
  One, they will be made much simpler.  The goal will be to find a minimal set of operations that will enable all desired plan features.
  
@@ -65, +68 @@

  propose to break relational and expression operators into different types, it will no longer be possible for a single visitor to span both types.  But we do not wish to
  prohibit this in all cases.
  
+ In the following code major members and methods of each class are shown.  Getters and setters are not shown unless they include functionality beyond simply getting and setting a
+ value.  
  
+ 
- New Operator class.  Note that the funciton which was previously called `visit` has been renamed `accept` to avoid confusion with the `visit` method in !PlanVisitor.
+ New Operator class.  Note that the function which was previously called `visit` has been renamed `accept` to avoid confusion with the `visit` method in !PlanVisitor.
  {{{
  
  package org.apache.pig.experimental.plan;
@@ -75, +81 @@

      
      protected String name;
      protected OperatorPlan plan; // plan that contains this operator
+     protected Map<String, Object> annotations;
  
-     public Operator(String n, OperatorPlan p) {...}
+     public Operator(String n, OperatorPlan p) { ... }
  
      /**
       * Accept a visitor at this node in the graph.
       * @param v Visitor to accept.
+      * @throws IOException 
       */
-     public abstract void accept(PlanVisitor v);
+     public abstract void accept(PlanVisitor v) throws IOException;
  
-     public String getName() { ... }
-     
      /**
-      * Get the plan associated with this operator.
-      * @return plan
+      * Add an annotation to a node in the plan.
+      * @param key string name of this annotation
+      * @param val value, as an Object
       */
-     public OperatorPlan getPlan() { ... }
+     public void annotate(String key, Object val) { ... }
+     
+     /**
+      * Look to see if a node is annotated.
+      * @param key string name of annotation to look for
+      * @return value of the annotation, as an Object, or null if the key is
+      * not present in the map.
+      */
+     public Object getAnnotation(String key) { ... } 
  
  }
  
- 
  }}}
  
+ New !OperatorPlan interface.  It has been made an interface so that different implementations can be used for representing actual plans and sub-graphs of plans inside the optimizer.
- New !OperatorPlan class.  Note the severe paring down of the number of operations.  Only simple add, remove, connect, disconnect.  All all operations are left to subclasses to implement 
+ Note the severe paring down of the number of operations.  Only simple add, remove, connect, disconnect.  All all operations are left to subclasses to implement 
  what makes sense for their plans.
  
  {{{
  
  package org.apache.pig.experimental.plan;
  
- public abstract class OperatorPlan {
+ public interface OperatorPlan {
- 
-     protected Set<Operator> ops;
-     protected PlanEdge fromEdges;
-     protected PlanEdge toEdges;
- 
-     public OperatorPlan() { ... }
      
      /**
- 	 * Get number of nodes in the plan.
+      * Get number of nodes in the plan.
       */
-     public int size() { ... }
+     public int size();
  
      /**
       * Get all operators in the plan that have no predecessors.
       * @return all operators in the plan that have no predecessors, or
       * an empty list if the plan is empty.
       */
-     public List<Operator> getRoots() { ... }
+     public List<Operator> getRoots();
  
      /**
       * Get all operators in the plan that have no successors.
       * @return all operators in the plan that have no successors, or
       * an empty list if the plan is empty.
       */
-     public List<Operator> getLeaves() { ... }
+     public List<Operator> getLeaves();
  
      /**
       * For a given operator, get all operators immediately before it in the
       * plan.
       * @param op operator to fetch predecessors of
-      * @return list of all operators imeediately before op, or an empty list
+      * @return list of all operators immediately before op, or an empty list
       * if op is a root.
       * @throws IOException if op is not in the plan.
       */
-     public List<Operator> getPredecessors(Operator op) throws IOException { ... }
+     public List<Operator> getPredecessors(Operator op) throws IOException;
      
      /**
       * For a given operator, get all operators immediately after it.
       * @param op operator to fetch successors of
-      * @return list of all operators imeediately after op, or an empty list
+      * @return list of all operators immediately after op, or an empty list
       * if op is a leaf.
       * @throws IOException if op is not in the plan.
       */
-     public List<Operator> getSuccessors(Operator op) throws IOException { ... }
+     public List<Operator> getSuccessors(Operator op) throws IOException;
  
      /**
       * Add a new operator to the plan.  It will not be connected to any
       * existing operators.
       * @param op operator to add
       */
-     public void add(Operator op) { ... }
+     public void add(Operator op);
  
      /**
       * Remove an operator from the plan.
@@ -163, +172 @@

       * @throws IOException if the remove operation attempts to 
       * remove an operator that is still connected to other operators.
       */
-     public void remove(Operator op) throws IOException { ... }
+     public void remove(Operator op) throws IOException;
-         
+     
      /**
- 	 * Connect two operators in the plan, controlling which position in the
+      * Connect two operators in the plan, controlling which position in the
- 	 * edge lists that the from and to edges are placed.
+      * edge lists that the from and to edges are placed.
- 	 * @param from Operator edge will come from
+      * @param from Operator edge will come from
- 	 * @param fromPos Position in the array for the from edge
+      * @param fromPos Position in the array for the from edge
- 	 * @param to Operator edge will go to
+      * @param to Operator edge will go to
- 	 * @param toPos Position in the array for the to edge
+      * @param toPos Position in the array for the to edge
       */
+     public void connect(Operator from, int fromPos, Operator to, int toPos);
-     public void connect(Operator from,
-                         int fromPos,
-                         Operator to,
-                         int toPos) { ... }
      
      /**
- 	 * Connect two operators in the plan.
+      * Connect two operators in the plan.
- 	 * @param from Operator edge will come from
+      * @param from Operator edge will come from
- 	 * @param to Operator edge will go to
+      * @param to Operator edge will go to
       */
-     public void connect(Operator from, Operator to) { ... }
+     public void connect(Operator from, Operator to);
      
      /**
       * Disconnect two operators in the plan.
@@ -193, +199 @@

       * to arrays.
       * @throws IOException if the two operators aren't connected.
       */
-     public Pair<Integer, Integer> disconnect(Operator from,
+     public Pair<Integer, Integer> disconnect(Operator from, Operator to) throws IOException;
-                                              Operator to) throws IOException { ... }
  
- }
  
+     /**
+      * Get an iterator of all operators in this plan
+      * @return an iterator of all operators in this plan
+      */
+     public Iterator<Operator> getOperators();
+ }
+     
  }}}
  
  There are no significant changes to !PlanVisitor and !PlanWalker other than the removal of generics.  With the change that only LOForeach now has inner plans, it isn't clear whether the
@@ -206, +217 @@

  == Changes to Logical Operators ==
  There will be a number of important changes to !LogicalOperators.
  
- First, as mentioned above relational opertors will be split into two disparate groups, relational and expression.
+ First, as mentioned above relational operators will be split into two disparate groups, relational and expression.
  
  Second, inner plans and expression plans will no longer hold any explicit references to outer plans.  At most, they will reference the operator which contains the inner plan.
  
  Third, operators will represent exactly one operation.
  
- Fourth, only LOForeach will have inner plans.  All other relational operators will
+ Fourth, only LOForeach will have inner plans.  All other relational operators will only have expressions.
- only have expressions.
  
+ Fifth, a new operator, LOInnerLoad will be introduced.  The sole purpose of this operator will be to act as a root in foreach's inner plans.  So, given a script like:
- Fifth, a new operator, called for now Identity but desparately seeking a better name,
- will be introduced.  The sole purpose of this operator will be to act as a root
- in foreach's inner plans.  So, given a script like:
  
  {{{
  A = load 'input';
@@ -229, +237 @@

  }
  }}}
  
- the foreach's inner plan will have two Identity operators, once for $0 and one for $1.
+ the foreach's inner plan will have two LOInnerLoad operators, one for $0 and one for $1.
- This will allow the C1 !LOGenerate operator to connect to another logical operator.
+ This will allow the C1 !LOGenerate operator to connect to another relational operator.
  
+ Sixth, in the past expression operators were used at places in inner plans, such as the `C1 = B1.$0;` above.  Relational operators will always be used in these places
- Sixth, in the past expression operators were used at places in inner plans, such as
- the `C1 = B1.$0;` above.  Relational operators will always be used in these places
- now.  LOGenerate (or perhaps a new operator if necessary) will be used in the place of
+ now.  LOGenerate (or perhaps a new operator if necessary) will be used in the place of these assignment operations instead.
- these assignment operations instead.
  
+ Seventh, in the past !LOForeach had multiple inner plans, one for each of its outputs.  That will no longer be the case.  !LOForeach will always have exactly one inner plan,
- Seventh, in the past !LOForeach had multiple inner plans, one for each of its outputs.
- That will no longer be the case.  !LOForeach will always have exactly one inner plan,
- which must terminate with a !LOGenerate.  That !LOGenerate will have expressions for
+ which must terminate with a !LOGenerate.  That !LOGenerate will have expressions for each of its outputs.
- each of its outputs.
  
+ All logical operators will have a schema.  This schema represents the format of the output for that operator.  The schema can be null, which indicates that the format of
- All logical operators will have a schema.  This schema represents the format of the
- output for that operator.  The schema can be null, which indicates that the format of
- the output for that operator is unknown.  In general the notion of unknownness in a
+ the output for that operator is unknown.  In general the notion of unknownness in a schema will be contagious.  Take for example:
- schema will be contagious.  Take for example:
  
  {{{
  A = load 'file1' as (x: int, y: float);
@@ -254, +256 @@

  D = foreach C generate flatten(A), flatten(B);
  }}}
  
+ A will have a schema, since one is specified for it.  B will not have a schema, since one is not specified.  C will have a schema, because the schema of (co)group is always
+ known.  Note however that in C's schema, the bag A will have a schema, and the bag B will not.  This means that D will not have schema, because the output of flatten(B) is
+ not known.  If D is changed to be `D = foreach C generate flatten(A);` then D will have a schema, since the format of flatten(A) is known.
- A will have a schema, since one is specified for it.  B will not have a schema, since
- one is not specified.  C will have a schema, because the schema of (co)group is always
- known.  Note however that in C's schema, the bag A will have a schema, and the bag B
- will not.  This means that D will not have schema, because the output of flatten(B) is
- not known.  If D is changed to be `D = foreach C generate flatten(A);` then D will
- have a schema, since the format of flatten(A) is known.
  
- !LogicalPlan will contain `add` and `removeLogical` operations specificly designed for manipulating logical plans.  These will be the only operations supported on the plan.
+ !LogicalPlan will contain `add` and `removeLogical` operations specifically designed for manipulating logical plans.  These will be the only operations supported on the plan.
  
  {{{
  
@@ -278, +277 @@

   * operators.  That will be handle by the various Patchers.
   *
   */
- public class LogicalPlan extends OperatorPlan {
+ public class LogicalPlan extends BaseOperatorPlan {
      
      /**
       * Add a relational operation to the plan.
@@ -289, +288 @@

       * be in the plan.
       * @param after operator  that will be after the new operator.  This
       * operator should already be in the plan.  If after is null, then the
-      * new operator will be a root.
+      * new operator will be a leaf.
       * @throws IOException if add is already in the plan, or before or after
       * are not in the plan.
       */
@@ -305, +304 @@

       * be in the plan.
       * @param after operator  that will be after the new operator.  This
       * operator should already be in the plan.  If after is null, then the
-      * new operator will be a root.
+      * new operator will be a leaf.
       * @throws IOException if add is already in the plan, or before or after
       * are not in the plan.
       */
-     public void add(List<LogicalRelationalOperator> before,
+     public void add(LogicalRelationalOperator[] before,
                      LogicalRelationalOperator newOper,
                      LogicalRelationalOperator after) throws IOException { ... }
      
@@ -327, +326 @@

       */
      public void add(LogicalRelationalOperator before,
                      LogicalRelationalOperator newOper,
-                     List<LogicalRelationalOperator> after) throws IOException { ... }
+                     LogicalRelationalOperator[] after) throws IOException { ... }
      
      /**
       * Add a relational operation to the plan when the caller wants to control
@@ -343, +342 @@

       * @param afterFromPos Position in newOps's edges to connect after at.
       * @param after operator  that will be after the new operator.  This
       * operator should already be in the plan.  If after is null, then the
-      * new operator will be a root.
+      * new operator will be a leaf.
       * @throws IOException if add is already in the plan, or before or after
       * are not in the plan.
       */
@@ -366, +365 @@

          
  }
  
- 
  }}}
  
  A !LogicalRelationalOperator will be the logical representation of a relational operator (join, sort, etc.).
@@ -383, +381 @@

      
      protected LogicalSchema schema;
      protected int requestedParallelism;
-     protected String alias;
-     protected int lineNum;
+     protected String alias; // Needed only for error messages, not used in optimizer
+     protected int lineNum;  // Needed only for error messages, not used in optimizer
  
      /**
       * 
@@ -402, +400 @@

      public LogicalRelationalOperator(String name,
                                       OperatorPlan plan,
                                       int rp) { ... }
-     
+ 
      /**
       * Get the schema for the output of this relational operator.  This does
       * not merely return the schema variable.  If schema is not yet set, this
@@ -413, +411 @@

      abstract public LogicalSchema getSchema();
      
      /**
-      * Get the requestedParallelism for this operator.
-      * @return requestedParallelsim
+      * Reset the schema to null so that the next time getSchema is called
+      * the schema will be regenerated from scratch.
       */
+     public void resetSchema() { ... }
-     public int getRequestedParallelism() { ... }
-     
-     /**
-      * Get the alias of this operator.  That is, if the Pig Latin for this operator
-      * was 'X = sort W by $0' then the alias will be X.  For store and split it will
-      * be the alias being stored or split.  Note that because of this this alias
-      * is not guaranteed to be unique to a single operator.
-      * @return alias
-      */
-     public String getAlias() { ... }
-     
-     /**
-      * Get the line number in the submitted Pig Latin script where this operator
-      * occurred.
-      * @return line number
-      */
-     public int getLineNumber() { ... }
- 
  }
- 
+     
  }}}
  
  !LogicalSchema will be based on the existing Schema class.  It is hoped that this class can be greatly simplified.
@@ -455, +436 @@

  
  }}}
  
- Since we already have plan structure we will have a !LogicalExpressionPlan.  This has the advantage that !PlanVisitors will work with expression trees and we do not need to invent a separate
+ Since we already have a plan structure we will have a !LogicalExpressionPlan.  This has the advantage that !PlanVisitors will work with expression trees and we do not need to invent
- visitor hierarchy. What operations !LogicalExpressionPlan will need is not yet clear.
+ a separate visitor hierarchy.
  
- !LogicalExpressionOperators will have a data type (the type they return) and a unique identifier (uid).  The point of the uid is to allow the optimizer to track how expressions flow through
+ !LogicalExpressionOperators will have a data type (the type they return) and a unique identifier (uid).  The point of the uid is to allow the optimizer to track how expressions flow
- the tree.  So projection expressions will have the same uid as the expression they are projecting.  All other expressions will create a new uid, since they are changing the value of the
+ through the tree.  So projection expressions will have the same uid as the expression they are projecting.  All other expressions will create a new uid, since they are changing the
- expression.
+ value of the expression.  (Cast probably does not conform to this statement.  In general casts are movable like projects.  We need to think further about how casts fit into this
+ system.)
+ 
+ {{{
+ 
+ package org.apache.pig.experimental.logical.expression;
+ 
+ /**
+  * Logical representation of expression operators.  Expression operators have
+  * a data type and a uid.  Uid is a unique id for each expression. 
+  *
+  */
+ public abstract class LogicalExpression extends Operator {
+     
+     protected byte type;
+     protected long uid = -1;
+ 
+     static public long getNextUid() { ... }
+     
+     /**
+      * 
+      * @param name of the operator
+      * @param plan LogicalExpressionPlan this is part of
+      * @param b datatype of this expression
+      */
+     public LogicalExpression(String name, OperatorPlan plan, byte b) { ... }
+     
+     
+     /**
+      * Set the uid.  For most expressions this will get a new uid.
+      * ProjectExpression needs to override this and find its uid from its
+      * predecessor.
+      * @param currentOp Current LogicalRelationalOperator that this expression operator
+      * is attached to.  Passed so that projection operators can determine their uid.
+      * @throws IOException
+      */
+     public void setUid(LogicalRelationalOperator currentOp) throws IOException { ... }
+     
+ }
+ 
+ }}}
  
  Consider the following example:
  
@@ -505, +526 @@

  == Changes to the Optimizer ==
  The following changes will be made to the optimizer:
   1. Currently all rules are handed to the optimizer at once, and it iterates over them until none of the rules trigger or it reaches the maximum number of iterations.  This will be changed so that rules are collected into sets.  The optimizer will then iterate over rules in each set until none of the rules trigger or it reaches the maximum number of iterations.  The reason for this change will be made clear below.
-  1. Currently the plan itself has the knowledge of how to patch itself up after it is rearranged.  (For example, how to reconstruct schemas after a plan is changed.)  This will be changed so that instead the optimizer can register a number of listeners (aka observers) on the plan.  These listeners will then be invoked after each rule that modifies the plan.  In these way the plans themselves need not understand how to patch up changes made by an optimization rule.  Also as we expand the plans and they record more information, it is easy to add new listeners without having to interact with existing functionality.
+  1. Currently the plan itself has the knowledge of how to patch itself up after it is rearranged.  (For example, how to reconstruct schemas after a plan is changed.)  This will be changed so that instead the optimizer can register a number of [[http://en.wikipedia.org/wiki/Observer_pattern|listeners]] on the plan.  These listeners will then be invoked after each rule that modifies the plan.  In this way the plans themselves need not understand how to patch up changes made by an optimization rule.  Also as we expand the plans and they record more information, it is easy to add new listeners without having to interact with existing functionality.
   1. The Rule class will be merged with the existing !RuleMatcher class so that Rule takes on the functionality of matching.  This match routine will be written once in Rule and extensions of rule need not re-implement it.
+ 
+ 
+ The rewritten Rule class:
+ 
+ {{{
+ 
+ package org.apache.pig.experimental.plan.optimizer;
+ 
+ /**
+  * Rules describe a pattern of operators.  They also reference a Transformer.
+  * If the pattern of operators is found one or more times in the provided plan,
+  * then the optimizer will use the associated Transformer to transform the
+  * plan.
+  *
+  */
+ public abstract class Rule {
+ 
+     protected String name = null;
+     protected OperatorPlan pattern;
+     transient protected OperatorPlan currentPlan;
+     
+     /**
+      * Create this rule by using the default pattern that this rule provided
+      * @param n Name of this rule
+      */
+     public Rule(String n) { .. }
+     
+     /**
+      * @param n Name of this rule
+      * @param p Pattern to look for.
+      */
+     public Rule(String n, OperatorPlan p) { ... }
+ 
+     /**
+      * Build the pattern that this rule will look for
+      * @return  the pattern to look for by this rule
+      */
+     abstract protected OperatorPlan buildPattern();
+     
+     /**
+      * Get the transformer for this rule.  Abstract because the rule
+      * may want to choose how to instantiate the transformer.  
+      * This should never return a cached transformer, it should
+      * always return a fresh one with no state.
+      * @return Transformer to use with this rule
+      */
+     abstract public Transformer getNewTransformer(); 
+     
+     /**
+      * Search for all the sub-plans that matches the pattern
+      * defined by this rule. 
+      * @return A list of all matched sub-plans. The returned plans are
+      *        partial views of the original OperatorPlan. Each is a 
+      *        sub-set of the original plan and represents the same
+      *        topology as the pattern, but operators in the returned plan  
+      *        are the same objects as the original plan. Therefore, 
+      *        a call getPlan() from any node in the return plan would 
+      *        return the original plan.
+      *        
+      * @param plan the OperatorPlan to look for matches to the pattern
+      */
+     public List<OperatorPlan> match(OperatorPlan plan) {... }     
+ }
+ 
+ }}}
+ 
+ The mostly unchanged Transformer class:
+ 
+ {{{
+ 
+ package org.apache.pig.experimental.plan.optimizer;
+ 
+ public abstract class Transformer {
+ 
+     /**
+      * check if the transform should be done.  If this is being called then
+      * the pattern matches, but there may be other criteria that must be met
+      * as well.
+      * @param matched the sub-set of the plan that matches the pattern. This 
+      *        subset has the same graph as the pattern, but the operators
+      *        point to the same objects as the plan to be matched.
+      * @return true if the transform should be done.
+      * @throws IOException
+      */
+     public abstract boolean check(OperatorPlan matched) throws IOException;
+ 
+     /**
+      * Transform the tree
+      * @param matched the sub-set of the plan that matches the pattern. This 
+      *        subset has the same graph as the pattern, but the operators
+      *        point to the same objects as the plan to be matched.
+      * @throws IOException
+      */
+     public abstract void transform(OperatorPlan matched) throws IOException;
+     
+     /**
+      * Report what parts of the tree were transformed.  This is so that 
+      * listeners can know which part of the tree to visit and modify
+      * schemas, annotations, etc.  So any nodes that were removed need
+      * will not be in this plan, only nodes that were added or moved.
+      * @return OperatorPlan that describes just the changed nodes.
+      */
+     public abstract OperatorPlan reportChanges();
+ 
+ }
+     
+ }}}
+ 
+ The new !PlanTransformListener interface:
+ 
+ {{{
+ 
+ package org.apache.pig.experimental.plan.optimizer;
+ 
+ /**
+  * A listener class that patches up plans after they have been transformed.
+  */
+ public interface PlanTransformListener {
+     /**
+      * the listener that is notified after a plan is transformed
+      * @param fp  the full plan that has been transformed
+      * @param tp  a plan containing only the operators that have been transformed
+      * @throws IOException 
+      */
+     public void transformed(OperatorPlan fp, OperatorPlan tp) throws IOException;
+ 
+ }
+     
+ }}}
+ 
  
  The goal in the above changes is to radically simplify writing optimizer rules.  Consider a rule to push a filter above a join.
  
@@ -517, +668 @@

  D = filter C by (y > 0 or v > 0) and x > 0 and u > 0 and y > v;
  }}}
  
- In the current design, to push this filter, a rule must know how to split filters, how to push the parts that pushable, and reconstruct the filters of the parts that are not.  In the new
+ In the current design, to push this filter, a rule must know how to split filters, how to push the parts that pushable, and reconstruct the filters of the parts that are not.  In the
- proposal we can instead create three rules.  Rule 1 will only know how split filters.  Rule 2 will only know how to push them.  And Rule 3 will only know how to reconstitute them.  These rules
+ new proposal we can instead create three rules.  Rule 1 will only know how split filters.  Rule 2 will only know how to push them.  And Rule 3 will only know how to reconstitute them.
- can then be placed in separate sets, so that they do not interfere with each other.  So in this example, after Rule 1 has run, the script will conceptually look like
+ These rules can then be placed in separate sets, so that they do not interfere with each other.  So in this example, after Rule 1 has run, the script will conceptually look like
  
  {{{
  A = load 'file1' as (x, y);
@@ -531, +682 @@

  D = filter D3 by y > v;
  }}}
  
- Since Rule 1 will be run repeatedly it need not manage entirely splitting the filter.  It can be written to simply split one and, allowing the next iteration to split any subsequents ands.
+ Since Rule 1 will be run repeatedly it need not manage entirely splitting the filter.  It can be written to simply split one and, allowing the next iteration to split any subsequents
+ ands.
  
- After several iterations of Rule 2, the script will look like:
+ After Rule 2, the script will look like:
  
  {{{
  A = load 'file1' as (x, y);
@@ -545, +697 @@

  D = filter D1 by y > v;
  }}}
  
- Again, Rule 2 does need to do this in one pass.  It can concentrate on pushing filters past the join one at a time.  (Most likely there will be a Rule 2.1 that will handle swapping filters so that we can get D2 past D1 and then apply Rule 2 to push D2 before C.)
- 
  And finally, after Rule 3 has run, the script will look like:
  
  {{{
@@ -560, +710 @@

  
  Writing each of these rules will be much simpler than writing one large rule that must handle all three cases.
  
- After each of these rules modify the tree, listeners will be notified that the tree has changed.  The currently known listeners are one to reconstruct schemas based on the changes and one to
+ After each of these rules modify the tree, listeners will be notified that the tree has changed.  The currently known listeners are one to reconstruct schemas based on the changes and
- assign uids to expressions in the tree.
+ one to reconnect projections to the proper field in their predecessor.  After each run of a rule and operations by attached listeners the plan will be in a functionally correct state.
  
+ We have identified three operations we would like to prototype.  The first is pushing filters past joins.  The second pushing filters after foreach with a flatten.  These were chosen
+ because both involve schema altering operations where the rules have to decide when they can and cannot push and where the listeners have real work to do to rewrite schemas and
+ projections after a rule is run.  The third operation is pruning unnecessary fields from the load.  That is, if five fields are loaded but only three are used, the other two will be
+ pruned out.  This was chosen because it proved to be particularly difficult in the current framework and we wish to investigate whether it is doable in the proposed framework.
- As part of the prototype we need to identify a set of test case rules that we believe represent operations that will need to be done on the plan.  Currently we have the following in this list:
-  * Pushing filter past join
-  * Pushing filter above foreach with flatten
-  * Pushing filter above group
-  * Pushing filter above sort
-  * Pushing filter above split
-  * Pushing filter above union
-  * Pushing filter above cross
-  * Pushing foreach with flatten below cogroup, join, cross, union
-  * Combine limit and sort
-  * Combine load with stream
-  * Combine stream with store
-  * Combine two filters
-  * Combine two foreachs
-  * Combine two joins