hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Owen O'Malley (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-1230) Replace parameters with context objects in Mapper, Reducer, Partitioner, InputFormat, and OutputFormat classes
Date Tue, 05 Aug 2008 22:34:44 GMT

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

Owen O'Malley commented on HADOOP-1230:
---------------------------------------

Alejandro,
  You seem to be looking at the wrong code. I committed into trunk the current version of
the proposed API. I still think that adding methods to the mapper is *far* more natural than
making a wrapping output context. The two approaches would look like:

*Option 1*

works for both subclasses that override run and/or the map methods

{code}
class MultipleOutputMapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
  extends Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {
  protected <KEY,VALUE> void collect(String output, KEY key, VALUE value) throws IOException
{...}
}
{code}

*Option 2*

would only work with classes that override the map method

{code}
public abstract class MultipleOutputMapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
  extends Mapper<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {

  protected MultipleOutputContext extends Context {
     protected MultipleOutputContext(Context outerContext) { ... }
     void collect(String output, KEY key, VALUE value) throws IOException {...}
  }

  protected void setup(MultipleOutputContext context
                              ) throws IOException, InterruptedException {
  }

  protected abstract void map(KEYIN key, VALUEIN value, MultipleOutputContext context
                                          ) throws IOException, InterruptedException;

  protected void cleanup(MultipleOutputContext context
                                  ) throws IOException, InterruptedException {
  }

  public void run(Context outerContext) throws IOException {
     MultipleOutputContext context = new MultipleOutputContext(outerContext);
    setup(context);
    KEYIN key = context.nextKey(null);
    VALUEIN value = null;
    while (key != null) {
      value = context.nextValue(value);
      map(key, value, context);
      key = context.nextKey(key);
    }
    cleanup(context);
  }
}
{code}

Note that these are *NOT* overrides of Mapper.setup, map, and cleanup, but instead are overloads
of them.

I think that option 1 is cleaner, but either one should work.

> Replace parameters with context objects in Mapper, Reducer, Partitioner, InputFormat,
and OutputFormat classes
> --------------------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-1230
>                 URL: https://issues.apache.org/jira/browse/HADOOP-1230
>             Project: Hadoop Core
>          Issue Type: Improvement
>          Components: mapred
>            Reporter: Owen O'Malley
>            Assignee: Owen O'Malley
>         Attachments: context-objs-2.patch, context-objs-3.patch, context-objs.patch
>
>
> This is a big change, but it will future-proof our API's. To maintain backwards compatibility,
I'd suggest that we move over to a new package name (org.apache.hadoop.mapreduce) and deprecate
the old interfaces and package. Basically, it will replace:
> package org.apache.hadoop.mapred;
> public interface Mapper extends JobConfigurable, Closeable {
>   void map(WritableComparable key, Writable value, OutputCollector output, Reporter reporter)
throws IOException;
> }
> with:
> package org.apache.hadoop.mapreduce;
> public interface Mapper extends Closable {
>   void map(MapContext context) throws IOException;
> }
> where MapContext has the methods like getKey(), getValue(), collect(Key, Value), progress(),
etc.

-- 
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