You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pig.apache.org by "Alan Gates (JIRA)" <ji...@apache.org> on 2009/03/04 02:08:56 UTC

[jira] Created: (PIG-697) Proposed improvements to pig's optimizer

Proposed improvements to pig's optimizer
----------------------------------------

                 Key: PIG-697
                 URL: https://issues.apache.org/jira/browse/PIG-697
             Project: Pig
          Issue Type: Bug
          Components: impl
            Reporter: Alan Gates
            Assignee: Alan Gates


I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:

1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
will have no effect on them.

2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
function is:

{code}
    public final void optimize() throws OptimizerException {
        RuleMatcher matcher = new RuleMatcher();
        for (Rule rule : mRules) {
            if (matcher.match(rule)) {
                // It matches the pattern.  Now check if the transformer
                // approves as well.
                List<List<O>> matches = matcher.getAllMatches();
                for (List<O> match:matches)
                {
	                if (rule.transformer.check(match)) {
	                    // The transformer approves.
	                    rule.transformer.transform(match);
	                }
                }
            }
        }
    }
{code}

It would change to be:
{code}
    public final void optimize() throws OptimizerException {
        RuleMatcher matcher = new RuleMatcher();
        boolean sawMatch;
        int iterators = 0;
        do {
            sawMatch = false;
            for (Rule rule : mRules) {
                List<List<O>> matches = matcher.getAllMatches();
                for (List<O> match:matches) {
                    // It matches the pattern.  Now check if the transformer
                    // approves as well.
                    if (rule.transformer.check(match)) {
                        // The transformer approves.
                        sawMatch = true;
                        rule.transformer.transform(match);
                    }
                }
            }
            // Not sure if 1000 is the right number of iterations, maybe it
            // should be configurable so that large scripts don't stop too 
            // early.
        } while (sawMatch && numIterations++ < 1000);
    }
{code}

The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
big picture of the entire plan.

3) Add three calls to OperatorPlan:

{code}
/**
 * Swap two operators in a plan.  Both of the operators must have single
 * inputs and single outputs.
 * @param first operator
 * @param second operator
 * @throws PlanException if either operator is not single input and output.
 */
public void swap(E first, E second) throws PlanException {
    ...
}

/**
 * Push one operator in front of another.  This function is for use when
 * the first operator has multiple inputs.  The caller can specify
 * which input of the first operator the second operator should be pushed to.
 * @param first operator, assumed to have multiple inputs.
 * @param second operator, will be pushed in front of first
 * @param inputNum, indicates which input of the first operator the second
 * operator will be pushed onto.  Numbered from 0.
 * @throws PlanException if inputNum does not exist for first operator
 */
public void pushBefore(E first, E second, int inputNum) throws PlanException {
    ...
}

/**
 * Push one operator after another.  This function is for use when the second
 * operator has multiple outputs.  The caller can specify which output of the
 * second operator the first operator should be pushed to.
 * @param first operator, will be pushed after the second operator
 * @param second operator, assumed to have multiple outputs
 * @param outputNum indicates which output of the second operator the first 
 * operator will be pushed onto.  Numbered from 0.
 * @throws PlanException if outputNum does not exist for second operator
 */
public void pushAfter(E first, E second, int outputNum) throws PlanException {
    ...
}
{code}

The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
plan.

4) Add a new call to Operator:

{code}
/**
 * Make any necessary changes to a node based on a change of position in the
 * plan.  This allows operators to rewire their projections, etc. when they
 * are relocated in a plan.
 * @param oldPred Operator that was previously the predecessor.
 * @param newPred Operator thwas will now be the predecessor.
 * @throws PlanException
 */
public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
{code}

This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
around so that the operator has a chance to make any necessary changes.  

5) Add new calls to LogicalOperator and PhysicalOperator

{code}
/**
 * A struct detailing how a projection is altered by an operator.
 */
public class ProjectionMap {
    /**
     * Quick way for an operator to note that its input and output are the
     * same.
     */
    public boolean noChange;

    /**
     * Map of field changes, with keys being the output fields of the 
     * operator and values being the input fields.  Fields are numbered from
     * 0.  So for a foreach operator derived from
     * 'B = foreach A generate $0, $2, $3, udf($1)' 
     * would produce a mapping of 0->0, 1->2, 2->3
     */
    public Map<Integer, Integer> mappedFields;

    /**
     * List of fields removed from the input.  This includes fields that were
     * transformed, and thus are no longer the same fields.  Using the
     * example foreach given under mappedFields, this list would contain '1'.
     */
    public List<Integer> removedFields;

    /**
     * List of fields in the output of this operator that were created by this
     * operator.  Using the example foreach given under mappedFields, this list
     * would contain '3'.
     */
    public List<Integer> addedFields;
}

/**
 * Produce a map describing how this operator modifies its projection.
 * @returns ProjectionMap null indicates it does not know how the projection
 * changes, for example a join of two inputs where one input does not have
 * a schema.
 */
public abstract ProjectionMap getProjectionMap();

/**
 * Get a list of fields that this operator requires.  This is not necessarily
 * equivalent to the list of fields the operator projects.  For example,
 * a filter will project anything passed to it, but requires only the fields
 * explicitly referenced in its filter expression.
 * @return list of fields, numbered from 0.
 */
public abstract List<Integer> getRequiredFields();
{code}

These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.

6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
Operators {Foreach, Filter}
Edges {0->1}

But I don't know if it will match rules that look like:
Operators {Scan, Scan, Join}
Edges {0->2, 1->2}

For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.

Examples of types of rules that is optimizer could support:

1) Pushing filters in front of joins.
2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
6) Merge filters or foreachs where possible

In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
to push work into the combiner.




-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Assigned: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alan Gates reassigned PIG-697:
------------------------------

    Assignee: Santhosh Srinivasan  (was: Alan Gates)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Issue Comment Edited: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12693961#action_12693961 ] 

Santhosh Srinivasan edited comment on PIG-697 at 3/30/09 5:51 PM:
------------------------------------------------------------------

Problem: Find a sub-graph within a directed acyclic graph (DAG) aka pattern matching
==============================================================

For optimization, a common process is to find patterns in a graph and rewire the graph to have an optimized version of the pattern. The problem of finding a sub-graph within a graph is the well known problem, sub-graph isomorphism, a NP complete problem. Within the context of PIG, the problem is recast as finding a sub DAG within a DAG.

The problem is divided into multiple sub-problems of representing the pattern (sub DAG) and finding the pattern within the DAG.

Representing the pattern.
-------------------------------------

The objective is to use the same optimizer framework to implement rule specification across the board, i.e., logical plans, physical plans and map reduce plans. In order to facilitate this, a new plan called RulePlan will be designed. The RulePlan will subclass the OperatorPlan and will be typed on RuleNode.

A RuleNode will extend the Operator class and will be annoated with the following member variables:

   1. mNodeClass: A private member variable that is of type Class to denote the class of the node. E.g.: mNodeClass could be LOFilter, LOLoad, POFilter, etc.
   2. mNodeType: An enum that idenitifes a node to be a simple node, a multi node or a common node. A brief descriptions of the three kinds of nodes follow.
      * Simple node: A normal rule node in the rule plan
      * Multi node: A node that appears more than once in the rule plan
      * Common node: A node that is common to more than one path, i.e., has multiple incoming edges or multiple outgoing edges. 

Finding the pattern.
----------------------------

The existing RuleMatcher class uses dependency order or depth first order to traverse the graph. This ensures that a given node is not seen more than once. Currently, the RuleMatcher relies on the list of nodes (provided as input) and picks the first node in the list as the root of the pattern. With the rule plan approach, the algorithm will be modified to look for all the roots of the RulePlan.

For each node in the matched path, the number of edges per node should match the number of edges for the corresponding node in the RulePlan. In addition, instead of looking for one edge from each node, the RuleMatcher will look for all edges from a given node in the RulePlan. The implication of this change, is that each RulePlan should be self contained, i.e., there cannot be any dangling edges out of the roots and into the intermediate and leaf nodes in the RulePlan.

When a set of matches is found, the following algorithm will compute the matches:

{code}
if there is a common node then
	for all matches m
		for all remaining matches r
			if(common_nodes(m) == common_nodes(r)) then
				put list of roots of each match into final_match list
			end if
		end for
	end for
else
	all matches have been found
end if
{code}

      was (Author: sms):
    Problem: Find a sub-graph within a directed acyclic graph (DAG) aka pattern matching
==============================================================

For optimization, a common process is to find patterns in a graph and rewire the graph to have an optimized version of the pattern. The problem of finding a sub-graph within a graph is the well known problem, sub-graph isomorphism, a NP complete problem. Within the context of PIG, the problem is recast as finding a sub DAG within a DAG.

The problem is divided into multiple sub-problems of representing the pattern (sub DAG) and finding the pattern within the DAG.

Representing the pattern.
-------------------------------------

The objective is to use the same optimizer framework to implement rule specification across the board, i.e., logical plans, physical plans and map reduce plans. In order to facilitate this, a new plan called RulePlan will be designed. The RulePlan will subclass the OperatorPlan and will be typed on RuleNode.

A RuleNode will extend the Operator class and will be annoated with the following member variables:

   1. mNodeClass: A private member variable that is of type Class to denote the class of the node. E.g.: mNodeClass could be LOFilter, LOLoad, POFilter, etc.
   2. mNodeType: An enum that idenitifes a node to be a simple node, a multi node or a common node. A brief descriptions of the three kinds of nodes follow.
      * Simple node: A normal rule node in the rule plan
      * Multi node: A node that appears more than once in the rule plan
      * Common node: A node that is common to more than one path, i.e., has multiple incoming edges or multiple outgoing edges. 

Finding the pattern.
----------------------------

The existing RuleMatcher class uses dependency order or depth first order to traverse the graph. This ensures that a given node is not seen more than once. Currently, the RuleMatcher relies on the list of nodes (provided as input) and picks the first node in the list as the root of the pattern. With the rule plan approach, the algorithm will be modified to look for all the roots of the RulePlan.

For each node in the matched path, the number of edges per node should match the number of edges for the corresponding node in the RulePlan. In addition, instead of looking for one edge from each node, the RuleMatcher will look for all edges from a given node in the RulePlan. The implication of this change, is that each RulePlan should be self contained, i.e., there cannot be any dangling edges out of the roots and into the intermediate and leaf nodes in the RulePlan.

When a set of matches is found, the following algorithm will compute the matches:

if there is a common node then
	for all matches m
		for all remaining matches r
			if(common_nodes(m) == common_nodes(r)) then
				put list of roots of each match into final_match list
			end if
		end for
	end for
else
	all matches have been found
end if

  
> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711861#action_12711861 ] 

Alan Gates commented on PIG-697:
--------------------------------

Comments on OptimizerPhase3_parrt1.patch

Why does LOSplit say it requires no fields?  If the split has filter conditions then it seems like it would need those fields.

Shouldn't LOStream require all fields rather than none?  It seems like users will have written their scripts assuming that their stream executable gets all of the fields coming out of the previous operator.



> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711863#action_12711863 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

LOSplit is a no-op operator. LOSplitOutput is modeled after filter.

Fair comment about LOStream.  I will make this change and resubmit the patch.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710469#action_12710469 ] 

Hadoop QA commented on PIG-697:
-------------------------------

+1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12408159/OptimizerPhase2.patch
  against trunk revision 775340.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 3 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/46/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/46/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/46/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase3_part2_2.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12725261#action_12725261 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

Phase 4 part 1 patch has been committed.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12698481#action_12698481 ] 

Alan Gates commented on PIG-697:
--------------------------------

Patch looks good.  A few comments on comments.

It looks like some of the comments in the code haven't been updated to reflect the changes.  They still talk about expressing rules as a list of nodes and edges, about only matching linear subsections of the graph, etc.  

Also, and more importantly, since the optimizer is someone complicated now I think it would be good to put a large comment in the package header for org.apache.pig.impl.plan.optimizer.  This comment should contain a basic outline of the optimizer design, including stuff like how graph of OperatorRule and RulePlan are used to match plans, the primitives used in graph transformations, etc.

I don't think either of these are big enough issues to prevent committing this patch.  They can both be included in the next patch.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Daniel Dai (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12740363#action_12740363 ] 

Daniel Dai commented on PIG-697:
--------------------------------

Two comments for Optimizer_Phase5.patch:
1. We can remove LOFRJoin.java, it is no longer in use
2. Remove comment "// For skewed join, add a local rearrange operator to the plan" in LogToPhyTranslationVisitor.java, both skewed join and regular join will do that, this comment is misleading.

Other part of the patch is good. 

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: Optimizer_Phase5.patch, OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase4_part2.patch

Attached patch introduces a new rule to push foreach with flatten down the tree, i.e., delay for each with faltten execution to reduce the number of records. A caveat here: This rule optimizes only one foreach with flatten input per cross/fragment replicate join. A new rule is required for binary, ternary, etc foreach with flatten input per cross/fragment replicate join. The cause for this limitation is the local view of the optimizer as opposed to a global view.



> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12699109#action_12699109 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

Patch has been committed.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710473#action_12710473 ] 

Alan Gates commented on PIG-697:
--------------------------------

+1 for OptimizerPhase2.patch

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12726684#action_12726684 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

Phase 4 part 2 patch has been committed

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

      Resolution: Fixed
    Hadoop Flags: [Reviewed]
          Status: Resolved  (was: Patch Available)

All optimizer related patches have been committed.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: Optimizer_Phase5.patch, OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12723939#action_12723939 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12411605/OptimizerPhase4_part1.patch
  against trunk revision 788174.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 12 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 1 new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/100/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/100/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/100/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712260#action_12712260 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

Patch OptimizerPhase3_part-1.patch has been committed.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711912#action_12711912 ] 

Hadoop QA commented on PIG-697:
-------------------------------

+1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12408756/OptimizerPhase3_parrt1-1.patch
  against trunk revision 776106.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 6 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/53/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/53/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/53/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase3_parrt1.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12726566#action_12726566 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

1. Removing added fields from the flattened set.

The flattened set is the set of all flattened columns. It can contain mapped and added fields. In order to remove the added fields from this set, the removeAll method is used.

2. Comments on why the rule applies only to Order, Cross and Join

Will add these comments.

3. Removing code in LOForEach for flattening a bag with unknown schema

The code that I removed was redundant and also had a bug. The check for a field getting mapped was neglected in one case. After I added the check, the code for the if and the else was identical. I removed the redundant code and made it simpler.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase3_parrt1.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710180#action_12710180 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12408159/OptimizerPhase2.patch
  against trunk revision 775340.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 3 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    -1 core tests.  The patch failed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/45/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/45/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/45/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12700958#action_12700958 ] 

Alan Gates commented on PIG-697:
--------------------------------

+1 on Part2 of Phase 1 patch.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12715143#action_12715143 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

The graph operation pushAfter was added as a complementary operation to pushBefore. Currently, on the logical side, there are no concrete use cases for pushAfter. The only operator that truly supports multiple outputs is split. Our current model for split is to have an no-op split operator that has multiple successors, split outputs, each of which is the equivalent of a filter. The split output has inner plans which could have projection operators that hold references to the split's predecessor. 

When an operator is pushed after split, the operator will be placed between the split and split output. As a result, when rewire on split is called, the call is dispatched to the split output. The references in the split output after the rewire will now point to split's predecessor instead of pointing to the operator that was pushed after.

The intention of the pushAfter in the case of a split is to push it after the split output. However, the generic pushAfter operation does not distinguish between split and split output. A possible way out is to override this method in the logical plan and duplicate most of the code in the OperatorPlan and add new code to handle split.

As of now, the pushAfter will not be used in the logical layer.


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: Open)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12726601#action_12726601 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

-1 javac. The applied patch generated 250 javac compiler warnings (more than the trunk's current 248 warnings).

The additional 2 compiler warning messages are related to type inference. At this point these messages are harmless. 

-1 javac. The applied patch generated 250 javac compiler warnings (more than the trunk's current 248 warnings).

Dodgy warning:
The find bug warnings are harmless, there is an  explicit check for null to print null as opposed to the contents of the object.  

Correctness warning:
There are checks in place to ensure that the variable can never be null.


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase2.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase1_part2.patch

Part2 of the Phase 1 patch. This patch adds the graph operators swap, pushBefore and pushAfter. In addition unit test cases have been added to cover the use of these new operators.

All unit tests pass.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722177#action_12722177 ] 

Hudson commented on PIG-697:
----------------------------

Integrated in Pig-trunk #480 (See [http://hudson.zones.apache.org/hudson/job/Pig-trunk/480/])
    : Proposed improvements to pig's optimizer (sms)


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12711480#action_12711480 ] 

Hadoop QA commented on PIG-697:
-------------------------------

+1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12408636/OptimizerPhase3_parrt1.patch
  against trunk revision 776106.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 6 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    +1 findbugs.  The patch does not introduce any new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/51/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/51/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/51/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase3_part2_1.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "David Ciemiewicz (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12697146#action_12697146 ] 

David Ciemiewicz commented on PIG-697:
--------------------------------------

Some thoughts on optimization problems and patterns from SQL and coding Pig and my desire for a higher level version of Pig than we have today.

I know this may come off as "distraction" but hopefully you'll have some time to hear me out.

* after a conversation with Santhosh about the SQL to Pig translation work 
* multiple issues I have countered with nested foreach statements including redundant function execution 
* nested FOREACH statement "assignment" computation bugs 
* hand coding chains of foreach statements so I can get the Algebraic combiner to kick 
* hand coding chains of foreach statements and grouping statements rather than using a single statement

I think I might have stumbled on a potentially improved model for Pig to Pig execution plan generation:

{code}
            High Level Pig to Low Level Pig translation
{code}

I think this would potentially benefit the SQL to Pig efforts and provide for programmer coding efficiency in Pig as well.

This will be a bit protracted, but I hope you have some time to consider it.

Take the following SQL idiom that the SQL to Pig translator will need to support:

{code}
            select
                        EXP(AVG(LN(time+0.1))) as geomean_time
            from
                        events
            where
                        time is not null and
                        time >= 0;
{code}

In "high level pig", I have wanted to code this as"
 
{code}
            A = load 'events' using PigStorage() as ( time: int );
            B = filter A by time is not null and time >= 0;
            C = group B all;
            D = foreach C generate EXP(AVG(LN(B.time+0.1))) as geomean_time;
{code}

In fact, this would seem to provide a nice translation path from SQL to "low level pig" via "high level pig".

Unfortunately, this won't work.  We developers must write Pig scripts at a lower level and break all of this apart into various steps.

An additional issue is that, because of some, um, workarounds, in the execution plan optimizations, the combiner won't kick in if we don't do further steps.

So the most "performant" version of the desired pig script is the following really "low level pig" where D is broken into 3 steps, merging one with B and the remaining 2 steps as separate D steps:

 
{code}
            A = load 'events' using PigStorage() as ( time: int );
            B = filter A by time is not null and time >= 0;
            B = foreach A generate LOG(time+0.1) as log_time;
            C = group B all;
            D = foreach C generate group, AVG(B.log_time) as mean_log_time;
                                    -- note that group alias is required for Algebraic combiner to kick in
            D = foreach D generate EXP(mean_log_time) as geomean_time;
{code}

If we can figure out how to translate SQL into this last "low-level" set of statements, why couldn't we or shouldn't we have "high level pig" as well and permit more efficient code writing and optimization?


Next example

I do a bunch of nested intermediate computations in a nested FOREACH statement:

{code}
C = foreach C {
        curr_mean_log_timetonextevent = curr_sum_log_timetonextevent / (double)count;
        curr_meansq_log_timetonextevent = curr_sumsq_log_timetonextevent / (double)count;
        curr_var_log_timetonextevent = curr_meansq_log_timetonextevent - 
                        (curr_mean_log_timetonextevent * curr_mean_log_timetonextevent);
        curr_sterr_log_timetonextevent = math.SQRT(curr_var_log_timetonextevent / (double)count);
 

        curr_geomean_timetonextevent = math.EXP(curr_mean_log_timetonextevent);
        curr_geosterr_timetonextevent = math.EXP(curr_sterr_log_timetonextevent);
        curr_mean_timetonextevent = curr_sum_log_timetonextevent / (double)count;
        curr_meansq_timetonextevent = curr_sumsq_log_timetonextevent / (double)count;
        curr_var_timetonextevent = curr_meansq_timetonextevent - 
                        (curr_mean_timetonextevent * curr_mean_timetonextevent);

        curr_sterr_timetonextevent = math.SQRT(curr_var_timetonextevent / count);

        generate
            ...
{code}

The code for nested statements in Pig has been particularly problematic and buggy including problems such as:

* redundant execution of functions such as SUM, AVG
* nested function problems
* mathematical operator problems (illustrated in this bug)
* no type propagation
* the need to use AS clauses to name nested alias assignments projected in the GENERATE clauses

What if instead of trying to do all of these operations in some specialized execution code, what if this was treated as "high level" pig that translated all of these intermediate statements into two or more "low level" foreach expansions.

This isn't as wild as it seems because 9 times out of 10, the "workaround" that I have had to do is exactly that: I had to stop using nested foreach and instead break the code into two separate foreach statements chained together.

In other words I went from the above nested foreach statement that generated errors and didn't work to two hand coded foreach statements (or more) that did:

{code}
C = foreach C generate
            *,
            curr_sum_log_timetonextevent / (double)count as curr_mean_log_timetonextevent,
            curr_sumsq_log_timetonextevent / (double)count as curr_meansq_log_timetonextevent;

C = foreach C generate=
            *,
            curr_meansq_log_timetonextevent - 
                        (curr_mean_log_timetonextevent * curr_mean_log_timetonextevent)
                                                                                    as curr_var_log_timetonextevent;

C = foreach C generate
            *,
            math.SQRT(curr_var_log_timetonextevent / (double)count) as curr_sterr_log_timetonextevent;
{code}

This was the only way I could avoid the redundant computations and get the code actually work. Well, actually if I added casts at appropriate places, it also worked, but what a pain.

This would also have the advantage that alias names used in the nested "assignments" would actually propagate without an "as" clause in the subsequent generate statement.

I know this is a "brain fart" but it does have a time honored tradition in languages like C, C++, Lisp of using the language to "bootstrap" the language by translating from more "high level" idioms to less feature rich "low level" idioms.

It just seemed like a plausible way of speeding up both development of a SQL to Pig translator as well as allowing a more rapid transition of Pig to higher level idioms while correcting whole swaths of execution bugs and performance optimization issues as well.




> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

Re-submitting the patch as the test cases as reported by HadoopQA pass on the developer's box.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase2.patch

Attaching a new patch for Optimizer Phase 2. The previous patch did not include a newly added file.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12720851#action_12720851 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12410960/OptimizerPhase3_part2_3.patch
  against trunk revision 785450.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 18 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    -1 javac.  The applied patch generated 259 javac compiler warnings (more than the trunk's current 224 warnings).

    +1 findbugs.  The patch does not introduce any new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/90/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/90/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/90/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12726528#action_12726528 ] 

Alan Gates commented on PIG-697:
--------------------------------

A couple of questions and a comment on patch4-part2

I don't understand what the following code does:
{code}
            List<Integer> foreachAddedFields = foreachProjectionMap.getAddedFields();
            if(foreachAddedFields != null) {
                Set<Integer> foreachAddedFieldsSet = new HashSet<Integer>(foreachAddedFields);
                flattenedColumnSet.removeAll(foreachAddedFieldsSet);
            }
{code}

Why are you removing added fields from the flattened set?  Won't all flattened fields appear as added in the projection map?

I think it would be very helpful to insert some comments on why this rule only applies if the successor is an Order, Cross, or Join.  

Why was the code dealing with flattening a bag with an unknown schema removed from LOForeach?


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12693961#action_12693961 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

Problem: Find a sub-graph within a directed acyclic graph (DAG) aka pattern matching
==============================================================

For optimization, a common process is to find patterns in a graph and rewire the graph to have an optimized version of the pattern. The problem of finding a sub-graph within a graph is the well known problem, sub-graph isomorphism, a NP complete problem. Within the context of PIG, the problem is recast as finding a sub DAG within a DAG.

The problem is divided into multiple sub-problems of representing the pattern (sub DAG) and finding the pattern within the DAG.

Representing the pattern.
-------------------------------------

The objective is to use the same optimizer framework to implement rule specification across the board, i.e., logical plans, physical plans and map reduce plans. In order to facilitate this, a new plan called RulePlan will be designed. The RulePlan will subclass the OperatorPlan and will be typed on RuleNode.

A RuleNode will extend the Operator class and will be annoated with the following member variables:

   1. mNodeClass: A private member variable that is of type Class to denote the class of the node. E.g.: mNodeClass could be LOFilter, LOLoad, POFilter, etc.
   2. mNodeType: An enum that idenitifes a node to be a simple node, a multi node or a common node. A brief descriptions of the three kinds of nodes follow.
      * Simple node: A normal rule node in the rule plan
      * Multi node: A node that appears more than once in the rule plan
      * Common node: A node that is common to more than one path, i.e., has multiple incoming edges or multiple outgoing edges. 

Finding the pattern.
----------------------------

The existing RuleMatcher class uses dependency order or depth first order to traverse the graph. This ensures that a given node is not seen more than once. Currently, the RuleMatcher relies on the list of nodes (provided as input) and picks the first node in the list as the root of the pattern. With the rule plan approach, the algorithm will be modified to look for all the roots of the RulePlan.

For each node in the matched path, the number of edges per node should match the number of edges for the corresponding node in the RulePlan. In addition, instead of looking for one edge from each node, the RuleMatcher will look for all edges from a given node in the RulePlan. The implication of this change, is that each RulePlan should be self contained, i.e., there cannot be any dangling edges out of the roots and into the intermediate and leaf nodes in the RulePlan.

When a set of matches is found, the following algorithm will compute the matches:

if there is a common node then
	for all matches m
		for all remaining matches r
			if(common_nodes(m) == common_nodes(r)) then
				put list of roots of each match into final_match list
			end if
		end for
	end for
else
	all matches have been found
end if


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12721968#action_12721968 ] 

Alan Gates commented on PIG-697:
--------------------------------

Why is it that some Logical operators (LOCross, LOStream) don't have rewire implemented?

Near the end of ProjectFixerUpper.vist(POProject), you have a TODO about the walking.  We should figure out whether that is necessary or not, as doing visiting by the visit function and by the walker can result in double visiting.

Is there a need to add a clear concept to LogicalTransformer in order to clear state between calls to check, since each transformer will potentially be called multiple times now?

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12700979#action_12700979 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

Committed Part2 patch of Phase 1.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712409#action_12712409 ] 

Hudson commented on PIG-697:
----------------------------

Integrated in Pig-trunk #451 (See [http://hudson.zones.apache.org/hudson/job/Pig-trunk/451/])
    : Proposed improvements to pig's optimizer


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase2.patch

Phase 2 of the optimizer introduces projection maps for the relational operators.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase4_part1-1.patch

Fixed the findbugs issue. This problem was prevalent in other parts of existing code. Fixed that too. Attaching new patch with these changes.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase1.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment:     (was: OptimizerPhase4_part1.patch)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Giridharan Kesavan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Giridharan Kesavan updated PIG-697:
-----------------------------------

    Status: Patch Available  (was: Open)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12721977#action_12721977 ] 

Alan Gates commented on PIG-697:
--------------------------------

+1, looks good.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12726581#action_12726581 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12412291/OptimizerPhase4_part2.patch
  against trunk revision 790635.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 15 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    -1 javac.  The applied patch generated 250 javac compiler warnings (more than the trunk's current 248 warnings).

    -1 findbugs.  The patch appears to introduce 2 new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/107/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/107/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/107/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12722075#action_12722075 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

OptimizerPhase3_part2_3.patch has been committed.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_part2_2.patch

Attached patch fixes the findbug warning, and cleans up the sources by removing commented out code. The additional 35 compiler warning messages are related to type inference. At this point these messages are harmless.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12724699#action_12724699 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12411941/OptimizerPhase4_part1-1.patch
  against trunk revision 788174.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 12 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 2 new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/103/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/103/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/103/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12740523#action_12740523 ] 

Hudson commented on PIG-697:
----------------------------

Integrated in Pig-trunk #515 (See [http://hudson.zones.apache.org/hudson/job/Pig-trunk/515/])
    : Proposed improvements to pig's optimizer, Phase5


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: Optimizer_Phase5.patch, OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12720465#action_12720465 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12410859/OptimizerPhase3_part2_2.patch
  against trunk revision 785450.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 18 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    -1 javac.  The applied patch generated 259 javac compiler warnings (more than the trunk's current 224 warnings).

    -1 findbugs.  The patch appears to introduce 1 new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/89/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/89/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/89/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_parrt1-1.patch

Attaching patch incorporating the review comments.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase1.patch

Attaching a new patch that fixes a javadoc warning.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12724731#action_12724731 ] 

Alan Gates commented on PIG-697:
--------------------------------

+1 on the phase 4 part 1 patch.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: Optimizer_Phase5.patch

Attached patch removes references to LOFRJoin and replaces it with LOJoin. All the optimization rules and test cases now use LOJoin.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: Optimizer_Phase5.patch, OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12727222#action_12727222 ] 

Hudson commented on PIG-697:
----------------------------

Integrated in Pig-trunk #494 (See [http://hudson.zones.apache.org/hudson/job/Pig-trunk/494/])
    

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_part2_1.patch

Attached patch includes the following:

1. Implementation of rewire with a modified API
2. Changes to projection map to facilitate the use of rewire
3. Turns of the store/load optimization in multi-query execution
4. Unit tests for rewire and modifications to existing unit tests

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710974#action_12710974 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12408536/OptimizerPhase3_parrt1.patch
  against trunk revision 776106.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 6 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to introduce 1 new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/49/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/49/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/49/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase1.patch

Attached patch has changed the design and implementation of the sub-graph pattern matching. Now, a subgraph pattern can be specified instead of a list of nodes and some edges. The existing rule specification was changed to use the new framework. Additional test cases have been added to validate and verify the new framework.

In addition,  PlanPrinter a generic plan printing class has been added. In the future, existing plan printers for the various types of plans (Logical, Physical, MR, RulePlan) should be changed to extend the PlanPrinter and override required methods.

All unit test cases pass.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Alan Gates
>         Attachments: OptimizerPhase1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12709511#action_12709511 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12408150/OptimizerPhase2.patch
  against trunk revision 774582.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 3 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    +1 javac.  The applied patch does not increase the total number of javac compiler warnings.

    -1 findbugs.  The patch appears to cause Findbugs to fail.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    -1 core tests.  The patch failed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/40/testReport/
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/40/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12710552#action_12710552 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

OptimizerPhase2 committed.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12721975#action_12721975 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

1. Some operators do not have any internal state that requires rewiring. Examples of such operators include LOStream, LOCross, etc.

2. I think that the additional walking should be removed. I added a TODO as I was not sure why it was added in the first place.

3. Yes, it will be added as part of the next patch.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_part2_3.patch

Fixed the new findbugs warning.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_parrt1.patch

Attaching new patch that fixes the findbugs warning.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase4_part1.patch

Attached patch, implements the optimization rule for pushing filters up.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Alan Gates (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12712153#action_12712153 ] 

Alan Gates commented on PIG-697:
--------------------------------

+1 for latest rev of part 3.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_parrt1.patch

Part 1 of the Phase3 patch. It implements the requiredFields feature in all the relational operators. New unit tests have been added.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12725601#action_12725601 ] 

Hudson commented on PIG-697:
----------------------------

Integrated in Pig-trunk #490 (See [http://hudson.zones.apache.org/hudson/job/Pig-trunk/490/])
    : Proposed improvements to pig's optimizer (sms)


> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Daniel Dai (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12740365#action_12740365 ] 

Daniel Dai commented on PIG-697:
--------------------------------

Phase5 patch committed. Thanks Santhosh!

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: Optimizer_Phase5.patch, OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Giridharan Kesavan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Giridharan Kesavan updated PIG-697:
-----------------------------------

    Status: Open  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: In Progress  (was: Patch Available)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: Optimizer_Phase5.patch, OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch, OptimizerPhase4_part2.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12724704#action_12724704 ] 

Santhosh Srinivasan commented on PIG-697:
-----------------------------------------

The find bug warnings are harmless, there are explicit checks for null to print null as opposed to the contents of the object.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_3.patch, OptimizerPhase4_part1-1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Attachment: OptimizerPhase3_parrt1.patch

New patch that adds projection map and required fields to operators that were left out in the previous patch (limit, split, split output and streaming).

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12720326#action_12720326 ] 

Hadoop QA commented on PIG-697:
-------------------------------

-1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12410837/OptimizerPhase3_part2_1.patch
  against trunk revision 784333.

    +1 @author.  The patch does not contain any @author tags.

    +1 tests included.  The patch appears to include 18 new or modified tests.

    +1 javadoc.  The javadoc tool did not generate any warning messages.

    -1 javac.  The applied patch generated 259 javac compiler warnings (more than the trunk's current 224 warnings).

    -1 findbugs.  The patch appears to introduce 1 new Findbugs warnings.

    +1 release audit.  The applied patch does not increase the total number of release audit warnings.

    +1 core tests.  The patch passed core unit tests.

    +1 contrib tests.  The patch passed contrib unit tests.

Test results: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/86/testReport/
Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/86/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: http://hudson.zones.apache.org/hudson/job/Pig-Patch-minerva.apache.org/86/console

This message is automatically generated.

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1-1.patch, OptimizerPhase3_parrt1.patch, OptimizerPhase3_part2_1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (PIG-697) Proposed improvements to pig's optimizer

Posted by "Santhosh Srinivasan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/PIG-697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Santhosh Srinivasan updated PIG-697:
------------------------------------

    Status: Patch Available  (was: In Progress)

> Proposed improvements to pig's optimizer
> ----------------------------------------
>
>                 Key: PIG-697
>                 URL: https://issues.apache.org/jira/browse/PIG-697
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>            Reporter: Alan Gates
>            Assignee: Santhosh Srinivasan
>         Attachments: OptimizerPhase1.patch, OptimizerPhase1_part2.patch, OptimizerPhase2.patch, OptimizerPhase3_parrt1.patch
>
>
> I propose the following changes to pig optimizer, plan, and operator functionality to support more robust optimization:
> 1) Remove the required array from Rule.  This will change rules so that they only match exact patterns instead of allowing missing elements in the pattern.
> This has the downside that if a given rule applies to two patterns (say Load->Filter->Group, Load->Group) you have to write two rules.  But it has the upside that
> the resulting rules know exactly what they are getting.  The original intent of this was to reduce the number of rules that needed to be written.  But the
> resulting rules have do a lot of work to understand the operators they are working with.  With exact matches only, each rule will know exactly the operators it
> is working on and can apply the logic of shifting the operators around.  All four of the existing rules set all entries of required to true, so removing this
> will have no effect on them.
> 2) Change PlanOptimizer.optimize to iterate over the rules until there are no conversions or a certain number of iterations has been reached.  Currently the
> function is:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         for (Rule rule : mRules) {
>             if (matcher.match(rule)) {
>                 // It matches the pattern.  Now check if the transformer
>                 // approves as well.
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches)
>                 {
> 	                if (rule.transformer.check(match)) {
> 	                    // The transformer approves.
> 	                    rule.transformer.transform(match);
> 	                }
>                 }
>             }
>         }
>     }
> {code}
> It would change to be:
> {code}
>     public final void optimize() throws OptimizerException {
>         RuleMatcher matcher = new RuleMatcher();
>         boolean sawMatch;
>         int iterators = 0;
>         do {
>             sawMatch = false;
>             for (Rule rule : mRules) {
>                 List<List<O>> matches = matcher.getAllMatches();
>                 for (List<O> match:matches) {
>                     // It matches the pattern.  Now check if the transformer
>                     // approves as well.
>                     if (rule.transformer.check(match)) {
>                         // The transformer approves.
>                         sawMatch = true;
>                         rule.transformer.transform(match);
>                     }
>                 }
>             }
>             // Not sure if 1000 is the right number of iterations, maybe it
>             // should be configurable so that large scripts don't stop too 
>             // early.
>         } while (sawMatch && numIterations++ < 1000);
>     }
> {code}
> The reason for limiting the number of iterations is to avoid infinite loops.  The reason for iterating over the rules is so that each rule can be applied multiple
> times as necessary.  This allows us to write simple rules, mostly swaps between neighboring operators, without worrying that we get the plan right in one pass.
> For example, we might have a plan that looks like:  Load->Join->Filter->Foreach, and we want to optimize it to Load->Foreach->Filter->Join.  With two simple
> rules (swap filter and join and swap foreach and filter), applied iteratively, we can get from the initial to final plan, without needing to understanding the
> big picture of the entire plan.
> 3) Add three calls to OperatorPlan:
> {code}
> /**
>  * Swap two operators in a plan.  Both of the operators must have single
>  * inputs and single outputs.
>  * @param first operator
>  * @param second operator
>  * @throws PlanException if either operator is not single input and output.
>  */
> public void swap(E first, E second) throws PlanException {
>     ...
> }
> /**
>  * Push one operator in front of another.  This function is for use when
>  * the first operator has multiple inputs.  The caller can specify
>  * which input of the first operator the second operator should be pushed to.
>  * @param first operator, assumed to have multiple inputs.
>  * @param second operator, will be pushed in front of first
>  * @param inputNum, indicates which input of the first operator the second
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if inputNum does not exist for first operator
>  */
> public void pushBefore(E first, E second, int inputNum) throws PlanException {
>     ...
> }
> /**
>  * Push one operator after another.  This function is for use when the second
>  * operator has multiple outputs.  The caller can specify which output of the
>  * second operator the first operator should be pushed to.
>  * @param first operator, will be pushed after the second operator
>  * @param second operator, assumed to have multiple outputs
>  * @param outputNum indicates which output of the second operator the first 
>  * operator will be pushed onto.  Numbered from 0.
>  * @throws PlanException if outputNum does not exist for second operator
>  */
> public void pushAfter(E first, E second, int outputNum) throws PlanException {
>     ...
> }
> {code}
> The rules in the optimizer can use these three functions, along with the existing insertBetween(), replace(), and removeAndReconnect() calls to operate on the
> plan.
> 4) Add a new call to Operator:
> {code}
> /**
>  * Make any necessary changes to a node based on a change of position in the
>  * plan.  This allows operators to rewire their projections, etc. when they
>  * are relocated in a plan.
>  * @param oldPred Operator that was previously the predecessor.
>  * @param newPred Operator thwas will now be the predecessor.
>  * @throws PlanException
>  */
> public abstract void rewire(Operator oldPred, Operator newPred) throws PlanException;
> {code}
> This method will be called by the swap, pushBefore, pushAfter, insertBetween, replace, and removeAndReconnect in OperatorPlan whenever an operator is moved
> around so that the operator has a chance to make any necessary changes.  
> 5) Add new calls to LogicalOperator and PhysicalOperator
> {code}
> /**
>  * A struct detailing how a projection is altered by an operator.
>  */
> public class ProjectionMap {
>     /**
>      * Quick way for an operator to note that its input and output are the
>      * same.
>      */
>     public boolean noChange;
>     /**
>      * Map of field changes, with keys being the output fields of the 
>      * operator and values being the input fields.  Fields are numbered from
>      * 0.  So for a foreach operator derived from
>      * 'B = foreach A generate $0, $2, $3, udf($1)' 
>      * would produce a mapping of 0->0, 1->2, 2->3
>      */
>     public Map<Integer, Integer> mappedFields;
>     /**
>      * List of fields removed from the input.  This includes fields that were
>      * transformed, and thus are no longer the same fields.  Using the
>      * example foreach given under mappedFields, this list would contain '1'.
>      */
>     public List<Integer> removedFields;
>     /**
>      * List of fields in the output of this operator that were created by this
>      * operator.  Using the example foreach given under mappedFields, this list
>      * would contain '3'.
>      */
>     public List<Integer> addedFields;
> }
> /**
>  * Produce a map describing how this operator modifies its projection.
>  * @returns ProjectionMap null indicates it does not know how the projection
>  * changes, for example a join of two inputs where one input does not have
>  * a schema.
>  */
> public abstract ProjectionMap getProjectionMap();
> /**
>  * Get a list of fields that this operator requires.  This is not necessarily
>  * equivalent to the list of fields the operator projects.  For example,
>  * a filter will project anything passed to it, but requires only the fields
>  * explicitly referenced in its filter expression.
>  * @return list of fields, numbered from 0.
>  */
> public abstract List<Integer> getRequiredFields();
> {code}
> These calls will be called by optimizer rules to determine whether or not a swap can be done (for example, you can't swap two operators if the second one uses a
> field added by the first), and once the swap is done they will be used by rewire to understand how to map projections in the operators.
> 6)  It's not clear that the RuleMatcher, in its current form, will work with rules that are not linear.  That is, it matches rules that look like:
> Operators {Foreach, Filter}
> Edges {0->1}
> But I don't know if it will match rules that look like:
> Operators {Scan, Scan, Join}
> Edges {0->2, 1->2}
> For the optimizer to be able to determine join types and operations with splits, it will have to be able to do that.
> Examples of types of rules that is optimizer could support:
> 1) Pushing filters in front of joins.
> 2) Pushing foreachs with flattens (which thus greathly expand the data) down the tree past filters, joins, etc.
> 3) Pushing type casting used for schemas in loads down to the point where the field is actually used.
> 4) Deciding when to do fragment/replicate join or sort/merge join instead of the standard hash join.
> 5) The current optimizations:  pushing limit up the tree, making implicit splits explicit, merge load and stream where possible, using the combiner.
> 6) Merge filters or foreachs where possible
> In particular the combiner optimizer hopefully can be completely rewritten to use the optimizer framework to make decisions about how to rework physical plans
> to push work into the combiner.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.