[ 
https://issues.apache.org/jira/browse/HADOOP-3702?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alejandro Abdelnur updated HADOOP-3702:
---------------------------------------

    Description: 
On the same input, we usually need to run multiple Maps one after the other 
without no Reduce. We also have to run multiple Maps after the Reduce.

If all pre-Reduce Maps are chained together and run as a single Map a 
significant amount of Disk I/O will be avoided. 

Similarly all post-Reduce Maps can be chained together and run in the Reduce 
phase after the Reduce.


  was:
On the same input, we usually need to run multiple Maps one after the other 
without no Reduce. We also have to run multiple Maps after the Reduce.

If all pre-Reduce Maps are chained together and run as a single Map a 
significant amount of Disk I/O will be avoided. 

Similarly all post-Reduce Maps can be chained together and run in the Reduce 
phase after the Reduce.

This could be done with ChainMapper and ChainReducer classes that would manage 
the chain of Maps and they would override the OutputCollector to implement the 
chaining.

The Maps and Reduce that are part of the Chain are unware they are executed in 
a Chain, they receive records via the {{map}} and {{reduce}} methods and do the 
output via the {{OutputCollector}}.

The API would look something like:

{code:java}

public class ChainMapper implements Mapper {

  public static void addMapper(JobConf job, Class<? extends Mapper> klass, 
Properties mapperConf);
  ...
}

public class ChainReducer implements Reducer {

  public static void setReducer(JobConf job, Class<? extends Reducer> klass, 
Properties reducerConf);

  public static void addMapper(JobConf job, Class<? extends Mapper> klass, 
Properties mapperConf);
  ...
}

{code}

The {{Properties}} configuration passed to the {{Mapper}} and {{Reducer}} when 
setting them into the chain are injected into a copy of the job's 
configuration. This allows maps to be configured as usual without being aware 
that they are in a chain.



> add support for chaining Maps in a single Map and after a Reduce [M*/RM*]
> -------------------------------------------------------------------------
>
>                 Key: HADOOP-3702
>                 URL: https://issues.apache.org/jira/browse/HADOOP-3702
>             Project: Hadoop Core
>          Issue Type: New Feature
>          Components: mapred
>         Environment: all
>            Reporter: Alejandro Abdelnur
>            Assignee: Alejandro Abdelnur
>            Priority: Minor
>
> On the same input, we usually need to run multiple Maps one after the other 
> without no Reduce. We also have to run multiple Maps after the Reduce.
> If all pre-Reduce Maps are chained together and run as a single Map a 
> significant amount of Disk I/O will be avoided. 
> Similarly all post-Reduce Maps can be chained together and run in the Reduce 
> phase after the Reduce.

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

Reply via email to