hadoop-pig-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alan Gates (JIRA)" <j...@apache.org>
Subject [jira] Commented: (PIG-171) Top K
Date Thu, 22 May 2008 15:07:55 GMT

    [ https://issues.apache.org/jira/browse/PIG-171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12599042#action_12599042
] 

Alan Gates commented on PIG-171:
--------------------------------

Here's an initial stab at a design for LIMIT.

To begin with we will implement LIMIT n, where n is the number of rows to return.  If users
ask for an offset capability in the future (LIMIT m, n where m is
an offset and n number of rows) it's easy enough to add.

Limit will be implemented in the new pipeline rework, as the framework is in place there to
more easily add it and to allow the optimizer to work with it.

The following areas will need to be changed to work with LIMIT:

1) Syntax in QueryParser.jjt.  LIMIT will need to be supported as statement in the language.
 So it should now be possible to say:

B = limit A 100;
...

Unit tests for this should be added to TestLogicalPlanBuilder.

2) Add LOLimit operator to logical operators.  This should be straight forward, as limit will
not modify the schema of its input relation.

3) Add POLimit operator to physical operators.  getNext() will need to keep a count of the
number of times it's been called.  Once that count exceeds the
provided limit, it will simply return END_OF_PROCESSING instead of calling getNext on it's
predecessor.  A new set of unit tests will need to be added for
this operator.

4) LogToPhyTranslationVisitor and MRCompiler will need be changed to handle translation of
LOLimit and POLimit.  Unit tests should be added to
TestLogToPhyCompiler and TestMRCompiler.

5) The logical optimizer (not yet built) will need to include rules for handling pushing of
Limit to the extent possible.  The rules should look like:

Limit cannot be pushed in front of:
    LOCogroup
    LOFilter
    LOLoad
    LOSort
    LOSplit
    LOSplitOutput
    LOStream (once that is in)
   
Limit can be pushed in front of:
    LOForEach, if it does not have a flatten.

Limit can be duplicated, and the new instance pushed in front of an operator for the following
operators (that is, if you have X->limit, you can transform
that to limit->X->limit):
    LOCross
    LODistinct
    LOForEach when it has a flatten
    LOUnion

6) The MRCompiler will need to know that it can move limit into the combiner in certain cases.
 If limit is the first operation in the reduce, then it can be
replicated and placed in the combine in addition to being in the reduce.



> Top K
> -----
>
>                 Key: PIG-171
>                 URL: https://issues.apache.org/jira/browse/PIG-171
>             Project: Pig
>          Issue Type: New Feature
>            Reporter: Amir Youssefi
>            Assignee: Amir Youssefi
>
> Frequently, users are interested on Top results (especially Top K rows) . This can be
implemented efficiently in Pig /Map Reduce settings to deliver rapid results and low Network
Bandwidth/Memory usage.
>  
>  Key point is to prune all data on the map side and keep only small set of rows with
Top criteria . We can do it in Algebraic function (combiner) with multiple value output. Only
a small data-set gets out of mapper node.
> The same idea is applicable to solve variants of this problem:
>   - An Algebraic Function for 'Top K Rows'
>   - An Algebraic Function for 'Top K' values ('Top Rank K' and 'Top Dense Rank K')
>   - TOP K ORDER BY.
> Another words implementation is similar to combiners for aggregate functions but instead
of one value we get multiple ones. 
> I will add a sample implementation for Top K Rows and possibly TOP K ORDER BY to clarify
details.

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


Mime
View raw message