You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-dev@hadoop.apache.org by Torsten Curdt <tc...@apache.org> on 2010/06/10 16:37:23 UTC

Re: Cannot initialize JVM Metrics with processName

Moving this over from the users list. I've been digging a little
deeper. In MultipleOutputs there is

 private TaskAttemptContext getContext(String nameOutput) throws IOException {
    // The following trick leverages the instantiation of a record writer via
    // the job thus supporting arbitrary output formats.
    Job job = new Job(context.getConfiguration());
    job.setOutputFormatClass(getNamedOutputFormatClass(context, nameOutput));
    job.setOutputKeyClass(getNamedOutputKeyClass(context, nameOutput));
    job.setOutputValueClass(getNamedOutputValueClass(context, nameOutput));
    TaskAttemptContext taskContext =
      new TaskAttemptContextImpl(job.getConfiguration(),
                                 context.getTaskAttemptID());
    return taskContext;
  }

so for every reduce call it creates a new Job instance ...which
creates a new LocalJobRunner.
That does not sound like a good idea.

I've created a patch and attached it to

https://issues.apache.org/jira/browse/MAPREDUCE-1853

Seems to work fine.

cheers
--
Torsten


On Thu, Jun 10, 2010 at 15:22, Torsten Curdt <tc...@vafer.org> wrote:
> Hadoop 0.21 using the new API. All working.
> Then I try to use MultipleOutputs in my reducer:
>
>    private MultipleOutputs<Text, Text> mos;
>
>    protected void setup(Context context) throws IOException,
> InterruptedException {
>        mos = new MultipleOutputs<Text, Text>(context);
>    }
>
>    protected String generateFileName(Text k, Text v) {
>        return "mos";
>    }
>
>    public void reduce(Text key, Iterable<Text> values, Context
> context) throws IOException, InterruptedException {
>        mos.write("test", key, new Text(""), "test");
>        mos.write("test", key, new Text(""));
>    }
>
> And start it up
>
>    Job job = Job.getInstance(cluster);
>    job.setJarByClass(Test.class);
>    job.setInputFormatClass(MyInputFormat.class);
>    job.setMapperClass(FanoutMapper.class);
>    job.setMapOutputKeyClass(Text.class);
>    job.setMapOutputValueClass(Text.class);
>    job.setCombinerClass(FanoutReducer.class);
>    job.setReducerClass(FanoutReducer.class);
>    job.setOutputFormatClass(SequenceFileOutputFormat.class);
>    job.setOutputKeyClass(Text.class);
>    job.setOutputValueClass(Text.class);
>    SequenceFileOutputFormat.setOutputPath(job, new Path(outputPath +
> "/output"));
>    MultipleOutputs.addNamedOutput(job, "test",
> SequenceFileOutputFormat.class, Text.class, Text.class);
>
>    job.waitForCompletion(true);
>
> But suddenly I get TONS of
>
>  jvm.JvmMetrics: Cannot initialize JVM Metrics with
> processName=JobTracker, sessionId= - already initialized
>
> and it feels much slower. (Maybe just because of the logging?)
>
> What's going on?
>
> Also I am wondering how I can specify the path for the output.
> I want to use just the "test" output for another job.
>
> cheers
> --
> Torsten
>