You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by Sandy <sn...@gmail.com> on 2008/09/24 02:40:40 UTC
setting a different input/output class for combiner function than map and reduce functions
Hi,
I'm trying to refine my map reduce algorithm to run faster, but I ran into a
little bit of trouble.
In my main, I have the following parameters set for my conf:
conf.setOutputKeyClass(Text.class);
conf.setOutputValueClass(LongWritable.class);
originally, this was quite ok, since my input to my map phase was key =
LongWritable value = Text,
and the output of my map phase was key = Text, value = LongWritable.
Similarily, the input to my reduce phase was key = Text, value =
iterator<LongWritable>
and the output was key = Text, value = LongWritable.
I just wrote a combiner class to try and speed things up. However, now I
want to do something like the following:
==map phase==
input: key = LongWritable value = Text,
output: key = Text, value = Longwritable
==combiner==
input: key = Text, value = iterator<LongWritable>
output: key = Text, value = Text
==reduce phase==
input: key = Text, value = iterator<Text>
output: key = Text, value = LongWritable
According to the documentation I saw, I shouldn't have to change what I have
set in main(), since the key type of the output of the job is Text, and the
value type is still LongWritable.
However, when I compile and try to run the code, I get the following:
08/09/23 18:35:45 INFO mapred.JobClient: Task Id :
task_200809222115_0004_m_000000_0, Status : FAILED
java.io.IOException: wrong value class: org.apache.hadoop.io.Text is not
class org.apache.hadoop.io.LongWritable
This tells me that Hadop doesn't like the fact that my output at the end of
the combiner is of type Text, which also tells me that I mustn't have set
the proper confs or something along those lines. However, I don't see
anything in the JobConf class that would let me set the output of the
combiner. I see a setMapOutputValueClass, but no setCombinerOutputValueClass
function.
Could someone point me in the right direction please?
Thanks in advance,
-SM
Re: setting a different input/output class for combiner function than map and reduce functions
Posted by Sandy <sn...@gmail.com>.
Thanks for the responses!
-SM
On Tue, Sep 23, 2008 at 8:11 PM, Owen O'Malley <om...@apache.org> wrote:
> On Tue, Sep 23, 2008 at 5:40 PM, Sandy <sn...@gmail.com> wrote:
>
> >
> > I just wrote a combiner class to try and speed things up. However, now I
> > want to do something like the following:
> > ==map phase==
> > input: key = LongWritable value = Text,
> > output: key = Text, value = Longwritable
> >
> > ==combiner==
> > input: key = Text, value = iterator<LongWritable>
> > output: key = Text, value = Text
>
>
> The input and output types for the combiner *must* be the same. The
> combiner
> may be applied 0, 1, or many times between the map and the reduce. So,
> combiners must be:
> * not depend on being run exactly once
> * not have side effects
>
> InputFormat -> Map -> Combiner* -> Reduce -> OutputFormat
>
> Since the Combiner may run more than once, it can't do type
> transformations.
>
> -- Owen
>
Re: setting a different input/output class for combiner function than map and reduce functions
Posted by Owen O'Malley <om...@apache.org>.
On Sep 24, 2008, at 2:24 AM, Devaraj Das wrote:
> If you are on 0.18, it is possible to say that a combiner be invoked
> once
> per partition per spill. Do
> job.setCombineOnlyOnce(true);
However, that functionality was introduced for backwards compatibility
with versions prior to 0.18 and was removed from 0.19. The combiner
should be viewed as a hint to framework for how to reduce the
transient data size. Your application really shouldn't be doing
transformations in the combiner.
-- Owen
Re: setting a different input/output class for combiner function than
map and reduce functions
Posted by Devaraj Das <dd...@yahoo-inc.com>.
If you are on 0.18, it is possible to say that a combiner be invoked once
per partition per spill. Do
job.setCombineOnlyOnce(true);
Or set the value of "mapred.combine.once" to true in your conf.
On 9/24/08 2:28 PM, "Palleti, Pallavi" <pa...@corp.aol.com> wrote:
> Can it be possible to ensure that a combiner must run only once?
>
> Thanks
> Pallavi
>
> -----Original Message-----
> From: owen.omalley@gmail.com [mailto:owen.omalley@gmail.com] On Behalf Of Owen
> O'Malley
> Sent: Wednesday, September 24, 2008 6:42 AM
> To: core-user@hadoop.apache.org
> Subject: Re: setting a different input/output class for combiner function than
> map and reduce functions
>
> On Tue, Sep 23, 2008 at 5:40 PM, Sandy <sn...@gmail.com> wrote:
>
>>
>> I just wrote a combiner class to try and speed things up. However, now I
>> want to do something like the following:
>> ==map phase==
>> input: key = LongWritable value = Text,
>> output: key = Text, value = Longwritable
>>
>> ==combiner==
>> input: key = Text, value = iterator<LongWritable>
>> output: key = Text, value = Text
>
>
> The input and output types for the combiner *must* be the same. The combiner
> may be applied 0, 1, or many times between the map and the reduce. So,
> combiners must be:
> * not depend on being run exactly once
> * not have side effects
>
> InputFormat -> Map -> Combiner* -> Reduce -> OutputFormat
>
> Since the Combiner may run more than once, it can't do type transformations.
>
> -- Owen
RE: setting a different input/output class for combiner function than map and reduce functions
Posted by "Palleti, Pallavi" <pa...@corp.aol.com>.
Can it be possible to ensure that a combiner must run only once?
Thanks
Pallavi
-----Original Message-----
From: owen.omalley@gmail.com [mailto:owen.omalley@gmail.com] On Behalf Of Owen O'Malley
Sent: Wednesday, September 24, 2008 6:42 AM
To: core-user@hadoop.apache.org
Subject: Re: setting a different input/output class for combiner function than map and reduce functions
On Tue, Sep 23, 2008 at 5:40 PM, Sandy <sn...@gmail.com> wrote:
>
> I just wrote a combiner class to try and speed things up. However, now I
> want to do something like the following:
> ==map phase==
> input: key = LongWritable value = Text,
> output: key = Text, value = Longwritable
>
> ==combiner==
> input: key = Text, value = iterator<LongWritable>
> output: key = Text, value = Text
The input and output types for the combiner *must* be the same. The combiner
may be applied 0, 1, or many times between the map and the reduce. So,
combiners must be:
* not depend on being run exactly once
* not have side effects
InputFormat -> Map -> Combiner* -> Reduce -> OutputFormat
Since the Combiner may run more than once, it can't do type transformations.
-- Owen
Re: setting a different input/output class for combiner function than map and reduce functions
Posted by Owen O'Malley <om...@apache.org>.
On Tue, Sep 23, 2008 at 5:40 PM, Sandy <sn...@gmail.com> wrote:
>
> I just wrote a combiner class to try and speed things up. However, now I
> want to do something like the following:
> ==map phase==
> input: key = LongWritable value = Text,
> output: key = Text, value = Longwritable
>
> ==combiner==
> input: key = Text, value = iterator<LongWritable>
> output: key = Text, value = Text
The input and output types for the combiner *must* be the same. The combiner
may be applied 0, 1, or many times between the map and the reduce. So,
combiners must be:
* not depend on being run exactly once
* not have side effects
InputFormat -> Map -> Combiner* -> Reduce -> OutputFormat
Since the Combiner may run more than once, it can't do type transformations.
-- Owen
Re: setting a different input/output class for combiner function than map and reduce functions
Posted by Arun C Murthy <ac...@yahoo-inc.com>.
> ==map phase==
> input: key = LongWritable value = Text,
> output: key = Text, value = Longwritable
>
> ==combiner==
> input: key = Text, value = iterator<LongWritable>
> output: key = Text, value = Text
>
The combiner is a pure optimization and *cannot* change the output
types of the map i.e. the combiner output _must_ be <Text,
LongWritable>.
> ==reduce phase==
> input: key = Text, value = iterator<Text>
> output: key = Text, value = LongWritable
>
You have a mismatch here too - the map-output types *must* be <Text,
Text> if your application needs the above. Your application is
assuming that the Combiner will change the types - this is an invalid
assumption.
The way around this is to get your map-output types to be <Text, Text>.
Arun