Return-Path: Delivered-To: apmail-hadoop-mapreduce-user-archive@minotaur.apache.org Received: (qmail 98801 invoked from network); 19 Jun 2010 17:02:01 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 19 Jun 2010 17:02:01 -0000 Received: (qmail 38530 invoked by uid 500); 19 Jun 2010 17:02:00 -0000 Delivered-To: apmail-hadoop-mapreduce-user-archive@hadoop.apache.org Received: (qmail 38434 invoked by uid 500); 19 Jun 2010 17:01:59 -0000 Mailing-List: contact mapreduce-user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: mapreduce-user@hadoop.apache.org Delivered-To: mailing list mapreduce-user@hadoop.apache.org Received: (qmail 38426 invoked by uid 99); 19 Jun 2010 17:01:59 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 19 Jun 2010 17:01:59 +0000 X-ASF-Spam-Status: No, hits=3.6 required=10.0 tests=AWL,FREEMAIL_ENVFROM_END_DIGIT,FREEMAIL_FROM,HTML_MESSAGE,RCVD_IN_DNSWL_NONE,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of lordjoe2000@gmail.com designates 209.85.210.48 as permitted sender) Received: from [209.85.210.48] (HELO mail-pz0-f48.google.com) (209.85.210.48) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 19 Jun 2010 17:01:54 +0000 Received: by pzk28 with SMTP id 28so170964pzk.35 for ; Sat, 19 Jun 2010 10:01:33 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:mime-version:received:received:in-reply-to :references:date:message-id:subject:from:to:content-type; bh=hFT5xMb/hRz1gSSb/gSfTRJT/qPByzb76QmVSlstN/0=; b=RQr7F1mMYubNOCWp+jAXnVb4AjE9Q5QbDcvv82/ENNLGSs/9ZLWhAykrFFlu2+SNCj v69tXtofExorf6dKF36QB/gC0m5vyz0bnYmrr7w7J+tG9XzF1JyXjY68TzZ6xvrygsUF Oo5uRN42ZHtLpexG+CqI69ggPZlrEPjmwNCLo= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; b=KN1PU7pU6u+CVLvAFa+f3HvqD1+fabh7U3tKN1CPMTrZUq2PiJ57S/CITQT8a/OYqH 5kMzO/4jfWCRj0Y3+3Qvns8ccSbDs9PF4AZvTimJmvHYo8/EdQVzjEujx904vnMhX/IG QMXLs7sOMMwFcQlAkCyGhDfmMUgnMa0OGdB0E= MIME-Version: 1.0 Received: by 10.114.164.39 with SMTP id m39mr2133774wae.56.1276966893622; Sat, 19 Jun 2010 10:01:33 -0700 (PDT) Received: by 10.114.106.1 with HTTP; Sat, 19 Jun 2010 10:01:33 -0700 (PDT) In-Reply-To: References: Date: Sat, 19 Jun 2010 10:01:33 -0700 Message-ID: Subject: Re: Need help with exception when mapper emits different key class from reducer From: Steve Lewis To: mapreduce-user@hadoop.apache.org Content-Type: multipart/alternative; boundary=0016367f931b15943f0489650671 --0016367f931b15943f0489650671 Content-Type: text/plain; charset=ISO-8859-1 Wow - I cannot tell you how much I thank you - I totally missed the fact that the exception is thrown in the combiner since I was seeing the exception in the reducer - I always thought the combiner was called between the mapper and the reducer and not after the reducer - Also does this mean I should use null as a combiner or use a very generic combiner - especially for my real problem when there is no real combiner step On Fri, Jun 18, 2010 at 2:45 PM, Eric Sammer wrote: > This took me a full read through to figure out. The problem is that > you're using your reducer as a combiner and when it runs, the output > of the map stage then becomes the wrong type. > > In pseudo-visual-speak: > > Object, Int -> Map() -> MyText, Int -> Combine() -> YourText, Int -> > EXCEPTION! > > When using your reducer as a combiner, the reducer outputs *must* > match the map outputs. In other words, your combiner - which is > *optional* in the chain at Hadoop's pleasure - is changing the key > space. That's a no-no. In your case, you can't reuse your reducer as a > combiner. > > (The hint is in the exception: it's occurring in the combiner classes > in Hadoop.) > > Hope that helps. > > On Fri, Jun 18, 2010 at 2:09 PM, Steve Lewis > wrote: > > > > This class is a copy of a standard WordCount class with one critical > > exception > > Instead of the Mapper Emitting a Key of Type Text it emits a key of type > > MyText - s simple subclass of Text > > The reducer emits a different subclass of Text - YourText > > I say > > job.setMapOutputKeyClass(MyText.class); > > job.setMapOutputValueClass(IntWritable.class); > > job.setOutputKeyClass(YourText.class); > > job.setOutputValueClass(IntWritable.class); > > which should declare these classes directly and yet I get the following > > exception using hadoop 0.2 on a local box > > What am I doing wrong > > > > java.io.IOException: wrong key class: class > > org.systemsbiology.hadoop.CapitalWordCount$YourText is not class > > org.systemsbiology.hadoop.CapitalWordCount$MyText > > at org.apache.hadoop.mapred.IFile$Writer.append(IFile.java:164) > > at > > > org.apache.hadoop.mapred.Task$CombineOutputCollector.collect(Task.java:880) > > at > > > org.apache.hadoop.mapred.Task$NewCombinerRunner$OutputConverter.write(Task.java:1201) > > at > > > org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutputContext.java:80) > > at > > > org.systemsbiology.hadoop.CapitalWordCount$IntSumReducer.reduce(CapitalWordCount.java:89) > > > > package org.systemsbiology.hadoop; > > import com.lordjoe.utilities.*; > > import org.apache.hadoop.conf.*; > > import org.apache.hadoop.fs.*; > > import org.apache.hadoop.io.*; > > import org.apache.hadoop.mapreduce.*; > > import org.apache.hadoop.mapreduce.lib.input.*; > > import org.apache.hadoop.mapreduce.lib.output.*; > > import org.apache.hadoop.util.*; > > import java.io.*; > > import java.util.*; > > /** > > * org.systemsbiology.hadoop.CapitalWordCount > > */ > > public class CapitalWordCount { > > public static class YourText extends Text > > { > > public YourText() { > > } > > /** > > * Construct from a string. > > */ > > public YourText(final String string) { > > super(string); > > } > > } > > public static class MyText extends Text > > { > > public MyText() { > > } > > /** > > * Construct from a string. > > */ > > public MyText(final String string) { > > super(string); > > } > > > > } > > public static class TokenizerMapper > > extends Mapper { > > private final static IntWritable one = new IntWritable(1); > > private MyText word = new MyText(); > > public void map(Object key, Text value, Context context > > ) throws IOException, InterruptedException { > > StringTokenizer itr = new StringTokenizer(value.toString()); > > while (itr.hasMoreTokens()) { > > String s = itr.nextToken().toUpperCase(); > > s = dropNonLetters(s); > > if (s.length() > 0) { > > word.set(s); > > context.write(word, one); > > } > > } > > } > > } > > public static String dropNonLetters(String s) { > > StringBuilder sb = new StringBuilder(); > > for (int i = 0; i < s.length(); i++) { > > char c = s.charAt(i); > > if (Character.isLetter(c)) > > sb.append(c); > > } > > return sb.toString(); > > } > > public static class IntSumReducer > > extends Reducer { > > private IntWritable result = new IntWritable(); > > public void reduce(MyText key, Iterable values, > > Context context > > ) throws IOException, InterruptedException { > > int sum = 0; > > for (IntWritable val : values) { > > sum += val.get(); > > } > > result.set(sum); > > context.write(new YourText(key.toString()), result); > > } > > } > > public static class MyPartitioner extends Partitioner IntWritable> > > { > > /** > > * Get the partition number for a given key (hence record) given > the > > total > > * number of partitions i.e. number of reduce-tasks for the job. > > *

> > *

Typically a hash function on a all or a subset of the > key.

> > * > > * @param key the key to be partioned. > > * @param value the entry value. > > * @param numPartitions the total number of partitions. > > * @return the partition number for the key. > > */ > > @Override > > public int getPartition(Text key, IntWritable value, int > > numPartitions) { > > String s = key.toString(); > > if (s.length() == 0) > > return 0; > > char c = s.charAt(0); > > int letter = Character.toUpperCase(c) - 'A'; > > if (letter < 0 || letter > 26) > > return 0; > > return letter % numPartitions; > > } > > } > > > > /** > > * Force loading of needed classes to make > > */ > > public static final Class[] NEEDED = > > { > > org.apache.commons.logging.LogFactory.class, > > org.apache.commons.cli.ParseException.class > > }; > > > > public static final int DEFAULT_REDUCE_TASKS = 14; > > public static void main(String[] args) throws Exception { > > Configuration conf = new Configuration(); > > String[] otherArgs = new GenericOptionsParser(conf, > > args).getRemainingArgs(); > > // if (otherArgs.length != 2) { > > // System.err.println("Usage: wordcount "); > > // System.exit(2); > > // } > > Job job = new Job(conf, "word count"); > > job.setJarByClass(CapitalWordCount.class); > > job.setMapperClass(TokenizerMapper.class); > > job.setCombinerClass(IntSumReducer.class); > > job.setReducerClass(IntSumReducer.class); > > > > job.setMapOutputKeyClass(MyText.class); > > job.setMapOutputValueClass(IntWritable.class); > > job.setOutputKeyClass(YourText.class); > > job.setOutputValueClass(IntWritable.class); > > > > // added Slewis > > job.setNumReduceTasks(DEFAULT_REDUCE_TASKS); > > job.setPartitionerClass(MyPartitioner.class); > > if(otherArgs.length > 1) { > > FileInputFormat.addInputPath(job, new Path(otherArgs[0])); > > } > > String athString = otherArgs[otherArgs.length - 1]; > > File out = new File(athString); > > if (out.exists()) { > > FileUtilities.expungeDirectory(out); > > out.delete(); > > } > > Path outputDir = new Path(athString); > > > > FileOutputFormat.setOutputPath(job, outputDir); > > > > boolean ans = job.waitForCompletion(true); > > int ret = ans ? 0 : 1; > > System.exit(ret); > > } > > } > > -- > > Steven M. Lewis PhD > > Institute for Systems Biology > > Seattle WA > > > > > > -- > Eric Sammer > twitter: esammer > data: www.cloudera.com > -- Steven M. Lewis PhD Institute for Systems Biology Seattle WA --0016367f931b15943f0489650671 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable Wow - =A0I cannot tell you how much I thank you - I totally missed the fact= that the exception is thrown in the combiner since I was seeing the except= ion in the reducer - I always thought the=A0combiner=A0was called between t= he mapper and the reducer and not after=A0
the reducer -=A0
Also does this mean I should use null as a combi= ner or use a very generic=A0combiner=A0-=A0especially=A0for my real problem= when there is no real combiner step

On Fri, Jun 18, 2010 at 2:45 PM, Eric Sammer <esammer@cloudera.com> wrote:=
This took me a full read through to figure = out. The problem is that
you're using your reducer as a combiner and when it runs, the output of the map stage then becomes the wrong type.

In pseudo-visual-speak:

Object, Int -> Map() -> MyText, Int -> Combine() -> YourText, I= nt -> EXCEPTION!

When using your reducer as a combiner, the reducer outputs *must*
match the map outputs. In other words, your combiner - which is
*optional* in the chain at Hadoop's pleasure - is changing the key
space. That's a no-no. In your case, you can't reuse your reducer a= s a
combiner.

(The hint is in the exception: it's occurring in the combiner classes in Hadoop.)

Hope that helps.

On Fri, Jun 18, 2010 at 2:09 PM, Steve Lewis <lordjoe2000@gmail.com> wrote:
>
> This class is a copy of a standard WordCount class with one critical > exception
> Instead of the Mapper Emitting a Key of Type Text it emits a key of ty= pe
> MyText - s simple subclass of Text
> The reducer emits a different subclass of Text - YourText
> I say
> =A0=A0 =A0 =A0 =A0job.setMapOutputKeyClass(MyText.class);
> =A0=A0 =A0 =A0 =A0job.setMapOutputValueClass(IntWritable.class);
> =A0=A0 =A0 =A0 =A0job.setOutputKeyClass(YourText.class);
> =A0=A0 =A0 =A0 =A0job.setOutputValueClass(IntWritable.class);
> which should declare these classes directly =A0and yet I get the follo= wing
> exception using hadoop 0.2 on a local box
> What am I doing wrong
>
> java.io.IOException: wrong key class: class
> org.systemsbiology.hadoop.CapitalWordCount$YourText is not class
> org.systemsbiology.hadoop.CapitalWordCount$MyText
> at org.apache.hadoop.mapred.IFile$Writer.append(IFile.java:164)
> at
> org.apache.hadoop.mapred.Task$CombineOutputCollector.collect(Task.java= :880)
> at
> org.apache.hadoop.mapred.Task$NewCombinerRunner$OutputConverter.write(= Task.java:1201)
> at
> org.apache.hadoop.mapreduce.TaskInputOutputContext.write(TaskInputOutp= utContext.java:80)
> at
> org.systemsbiology.hadoop.CapitalWordCount$IntSumReducer.reduce(Capita= lWordCount.java:89)
>
> package org.systemsbiology.hadoop;
> import com.lordjoe.utilities.*;
> import org.apache.hadoop.conf.*;
> import org.apache.hadoop.fs.*;
> import org.apache.hadoop.io.*;
> import org.apache.hadoop.mapreduce.*;
> import org.apache.hadoop.mapreduce.lib.input.*;
> import org.apache.hadoop.mapreduce.lib.output.*;
> import org.apache.hadoop.util.*;
> import java.io.*;
> import java.util.*;
> /**
> =A0* =A0org.systemsbiology.hadoop.CapitalWordCount
> =A0*/
> public class CapitalWordCount {
> =A0=A0 =A0public static class YourText extends Text
> =A0=A0 =A0 =A0{
> =A0=A0 =A0 =A0 =A0 =A0public YourText() {
> =A0=A0 =A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0 =A0/**
> =A0=A0 =A0 =A0 =A0 =A0 * Construct from a string.
> =A0=A0 =A0 =A0 =A0 =A0 */
> =A0=A0 =A0 =A0 =A0 =A0public YourText(final String string) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0super(string);
> =A0=A0 =A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0}
> =A0=A0 =A0public static class MyText extends Text
> =A0=A0 =A0{
> =A0=A0 =A0 =A0 =A0public MyText() {
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0/**
> =A0=A0 =A0 =A0 =A0 * Construct from a string.
> =A0=A0 =A0 =A0 =A0 */
> =A0=A0 =A0 =A0 =A0public MyText(final String string) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0super(string);
> =A0=A0 =A0 =A0 =A0}
>
> =A0=A0 =A0}
> =A0=A0 =A0public static class TokenizerMapper
> =A0=A0 =A0 =A0 =A0 =A0 =A0extends Mapper<Object, Text, MyText, IntW= ritable> {
> =A0=A0 =A0 =A0 =A0private final static IntWritable one =3D new IntWrit= able(1);
> =A0=A0 =A0 =A0 =A0private MyText word =3D new MyText();
> =A0=A0 =A0 =A0 =A0public void map(Object key, Text value, Context cont= ext
> =A0=A0 =A0 =A0 =A0) throws IOException, InterruptedException {
> =A0=A0 =A0 =A0 =A0 =A0 =A0StringTokenizer itr =3D new StringTokenizer(= value.toString());
> =A0=A0 =A0 =A0 =A0 =A0 =A0while (itr.hasMoreTokens()) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0String s =3D itr.nextToken().toUpper= Case();
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0s =3D dropNonLetters(s);
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0if (s.length() > 0) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0word.set(s);
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0context.write(word, one); > =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0}
> =A0=A0 =A0public static String dropNonLetters(String s) {
> =A0=A0 =A0 =A0 =A0StringBuilder sb =3D new StringBuilder();
> =A0=A0 =A0 =A0 =A0for (int i =3D 0; i < s.length(); i++) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0char c =3D s.charAt(i);
> =A0=A0 =A0 =A0 =A0 =A0 =A0if (Character.isLetter(c))
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0sb.append(c);
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0return sb.toString();
> =A0=A0 =A0}
> =A0=A0 =A0public static class IntSumReducer
> =A0=A0 =A0 =A0 =A0 =A0 =A0extends Reducer<MyText, IntWritable, Your= Text, IntWritable> {
> =A0=A0 =A0 =A0 =A0private IntWritable result =3D new IntWritable(); > =A0=A0 =A0 =A0 =A0public void reduce(MyText key, Iterable<IntWritab= le> values,
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 Context context=
> =A0=A0 =A0 =A0 =A0) throws IOException, InterruptedException {
> =A0=A0 =A0 =A0 =A0 =A0 =A0int sum =3D 0;
> =A0=A0 =A0 =A0 =A0 =A0 =A0for (IntWritable val : values) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0sum +=3D val.get();
> =A0=A0 =A0 =A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0 =A0 =A0result.set(sum);
> =A0=A0 =A0 =A0 =A0 =A0 =A0context.write(new YourText(key.toString()), = result);
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0}
> =A0=A0 =A0public static class MyPartitioner extends Partitioner<Tex= t, IntWritable>
> {
> =A0=A0 =A0 =A0 =A0/**
> =A0=A0 =A0 =A0 =A0 * Get the partition number for a given key (hence r= ecord) given the
> total
> =A0=A0 =A0 =A0 =A0 * number of partitions i.e. number of reduce-tasks = for the job.
> =A0=A0 =A0 =A0 =A0 * <p/>
> =A0=A0 =A0 =A0 =A0 * <p>Typically a hash function on a all or a = subset of the key.</p>
> =A0=A0 =A0 =A0 =A0 *
> =A0=A0 =A0 =A0 =A0 * @param key =A0 =A0 =A0 =A0 =A0 the key to be part= ioned.
> =A0=A0 =A0 =A0 =A0 * @param value =A0 =A0 =A0 =A0 the entry value.
> =A0=A0 =A0 =A0 =A0 * @param numPartitions the total number of partitio= ns.
> =A0=A0 =A0 =A0 =A0 * @return the partition number for the <code>= key</code>.
> =A0=A0 =A0 =A0 =A0 */
> =A0=A0 =A0 =A0 =A0@Override
> =A0=A0 =A0 =A0 =A0public int getPartition(Text key, IntWritable value,= int
> numPartitions) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0String s =3D key.toString();
> =A0=A0 =A0 =A0 =A0 =A0 =A0if (s.length() =3D=3D 0)
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0return 0;
> =A0=A0 =A0 =A0 =A0 =A0 =A0char c =3D s.charAt(0);
> =A0=A0 =A0 =A0 =A0 =A0 =A0int letter =3D Character.toUpperCase(c) - &#= 39;A';
> =A0=A0 =A0 =A0 =A0 =A0 =A0if (letter < 0 || letter > 26)
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0return 0;
> =A0=A0 =A0 =A0 =A0 =A0 =A0return letter % numPartitions;
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0}
>
> =A0=A0 =A0/**
> =A0=A0 =A0 * Force loading of needed classes to make
> =A0=A0 =A0 */
> =A0=A0 =A0public static final Class[] NEEDED =3D
> =A0=A0 =A0 =A0 =A0 =A0 =A0{
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0org.apache.commons.logging.L= ogFactory.class,
> =A0=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0org.apache.commons.cli.Parse= Exception.class
> =A0=A0 =A0 =A0 =A0 =A0 =A0};
>
> =A0=A0 =A0public static final int DEFAULT_REDUCE_TASKS =3D 14;
> =A0=A0 =A0public static void main(String[] args) throws Exception { > =A0=A0 =A0 =A0 =A0Configuration conf =3D new Configuration();
> =A0=A0 =A0 =A0 =A0String[] otherArgs =3D new GenericOptionsParser(conf= ,
> args).getRemainingArgs();
> // =A0 =A0 =A0 =A0if (otherArgs.length !=3D 2) {
> // =A0 =A0 =A0 =A0 =A0 =A0System.err.println("Usage: wordcount &l= t;in> <out>");
> // =A0 =A0 =A0 =A0 =A0 =A0System.exit(2);
> // =A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0Job job =3D new Job(conf, "word count"); > =A0=A0 =A0 =A0 =A0job.setJarByClass(CapitalWordCount.class);
> =A0=A0 =A0 =A0 =A0job.setMapperClass(TokenizerMapper.class);
> =A0=A0 =A0 =A0 =A0job.setCombinerClass(IntSumReducer.class);
> =A0=A0 =A0 =A0 =A0job.setReducerClass(IntSumReducer.class);
>
> =A0=A0 =A0 =A0 =A0job.setMapOutputKeyClass(MyText.class);
> =A0=A0 =A0 =A0 =A0job.setMapOutputValueClass(IntWritable.class);
> =A0=A0 =A0 =A0 =A0job.setOutputKeyClass(YourText.class);
> =A0=A0 =A0 =A0 =A0job.setOutputValueClass(IntWritable.class);
>
> =A0=A0 =A0 =A0 =A0// added Slewis
> =A0=A0 =A0 =A0 =A0job.setNumReduceTasks(DEFAULT_REDUCE_TASKS);
> =A0=A0 =A0 =A0 =A0job.setPartitionerClass(MyPartitioner.class);
> =A0=A0 =A0 =A0 =A0if(otherArgs.length > 1) =A0 =A0{
> =A0=A0 =A0 =A0 =A0 =A0 =A0FileInputFormat.addInputPath(job, new Path(o= therArgs[0]));
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0String athString =3D otherArgs[otherArgs.length - 1]= ;
> =A0=A0 =A0 =A0 =A0File out =3D new File(athString);
> =A0=A0 =A0 =A0 =A0if (out.exists()) {
> =A0=A0 =A0 =A0 =A0 =A0 =A0FileUtilities.expungeDirectory(out);
> =A0=A0 =A0 =A0 =A0 =A0 =A0out.delete();
> =A0=A0 =A0 =A0 =A0}
> =A0=A0 =A0 =A0 =A0Path outputDir =3D new Path(athString);
>
> =A0=A0 =A0 =A0 =A0FileOutputFormat.setOutputPath(job, outputDir);
>
> =A0=A0 =A0 =A0 =A0boolean ans =3D job.waitForCompletion(true);
> =A0=A0 =A0 =A0 =A0int ret =3D ans ? 0 : 1;
> =A0=A0 =A0 =A0 =A0System.exit(ret);
> =A0=A0 =A0}
> }
> --
> Steven M. Lewis PhD
> Institute for Systems Biology
> Seattle WA
>



--
Eric Sammer
twitter: esammer
data: www.cloudera.co= m



--
Steven M. Lewis = PhD
Institute for Systems Biology
Seattle WA
--0016367f931b15943f0489650671--