hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Enis Soztutar (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-3702) add support for chaining Maps in a single Map and after a Reduce [M*/RM*]
Date Fri, 05 Sep 2008 09:39:45 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-3702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12628600#action_12628600
] 

Enis Soztutar commented on HADOOP-3702:
---------------------------------------

Ok, I confused option 3 as a derivative of 1, which clearly is a derivative of 2.
However the output classes can indeed be more specific than declared. An example of this is
a mapper which does not change the input key, and directly outputs it. 

The following code fragment, demonstrates that we should go with the option 2. 
{code}
  static class A  {
  }
  static class B extends A {
  }
  static class AMapper extends MapReduceBase implements Mapper<A, Text, A, Text> {
    @Override
    public void map(A key, Text value, OutputCollector<A, Text> output,
        Reporter reporter) throws IOException {
    }
  }
  static class BMapper extends MapReduceBase implements Mapper<B, Text, B, Text> {
    @Override
    public void map(B key, Text value, OutputCollector<B, Text> output,
        Reporter reporter) throws IOException {
    }
  }
  static {
    JobConf job = new JobConf();
    addMapper(true, job, AMapper.class, A.class, Text.class, A.class, Text.class, false, job);
    addMapper(true, job, AMapper.class, B.class, Text.class, B.class, Text.class, false, job);
    addMapper(true, job, BMapper.class, A.class, Text.class, A.class, Text.class, false, job);//should
fail
    addMapper(true, job, BMapper.class, B.class, Text.class, B.class, Text.class, false, job);
  }
{code}
 



> 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
>             Fix For: 0.19.0
>
>         Attachments: Hadoop-3702.patch, patch3702.txt, patch3702.txt, patch3702.txt,
patch3702.txt, patch3702.txt, patch3702.txt, patch3702.txt, patch3702.txt, patch3702.txt,
patch3702.txt, patch3702.txt, patch3702.txt
>
>
> 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.


Mime
View raw message