[ 
https://issues.apache.org/jira/browse/MAPREDUCE-1214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12789973#action_12789973
 ] 

Jeff Zhang commented on MAPREDUCE-1214:
---------------------------------------

Ankit,
I have tried to use counters in hadoop local mode, and it works. Also custom 
counters is supported in hadoop local mode. Hope I did not misunderstand your 
meaning.
Here's my code snippet and output
{code}
public class MapReduceExample extends Configured implements Tool {

        static class MyMapper extends Mapper<LongWritable, Text, LongWritable, 
Text> {
                public MyMapper(){
                        
                }
                
                protected void map(
                                LongWritable key,
                                Text value,
                                
org.apache.hadoop.mapreduce.Mapper<LongWritable, Text, LongWritable, 
Text>.Context context)
                                throws java.io.IOException, 
InterruptedException {
                        context.getCounter("mygroup", "jeff").increment(1);
                        context.write(key, value);
                };
        }

        @Override
        public int run(String[] args) throws Exception {
                Job job = new Job();
                job.setMapperClass(MyMapper.class);
                FileInputFormat.setInputPaths(job, new Path(args[0]));
                FileOutputFormat.setOutputPath(job, new Path(args[1]));

                job.waitForCompletion(true);
                return 0;
        }

        public static void main(String[] args) throws Exception {
                FileUtils.deleteDirectory(new File("data/output"));
                args = new String[] { "data/input", "data/output" };
                ToolRunner.run(new MapReduceExample(), args);
        }
}
{code}

{code}
2009-12-14 11:38:41,843 INFO  mapred.JobClient 
(JobClient.java:monitorAndPrintJob(1288)) -  map 100% reduce 100%
2009-12-14 11:38:41,843 INFO  mapred.JobClient 
(JobClient.java:monitorAndPrintJob(1343)) - Job complete: job_local_0001
2009-12-14 11:38:41,843 INFO  mapred.JobClient (Counters.java:log(514)) - 
Counters: 13
2009-12-14 11:38:41,843 INFO  mapred.JobClient (Counters.java:log(516)) -   
FileSystemCounters
2009-12-14 11:38:41,843 INFO  mapred.JobClient (Counters.java:log(518)) -     
FILE_BYTES_READ=26096
2009-12-14 11:38:41,843 INFO  mapred.JobClient (Counters.java:log(518)) -     
FILE_BYTES_WRITTEN=52792
2009-12-14 11:38:41,843 INFO  mapred.JobClient (Counters.java:log(516)) -   
mygroup
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
jeff=2
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(516)) -   
Map-Reduce Framework
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Reduce input groups=0
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Combine output records=0
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Map input records=2
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Reduce shuffle bytes=0
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Reduce output records=0
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Spilled Records=4
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Map output bytes=28
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Combine input records=0
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Map output records=2
2009-12-14 11:38:41,858 INFO  mapred.JobClient (Counters.java:log(518)) -     
Reduce input records=2
{code}


> Add support for counters in Hadoop Local Mode
> ---------------------------------------------
>
>                 Key: MAPREDUCE-1214
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-1214
>             Project: Hadoop Map/Reduce
>          Issue Type: Improvement
>            Reporter: Ankit Modi
>
> Currently there is no support for counters ( Records and Bytes written ) in 
> Hadoop Local Mode.
> Pig requires to provide counters to user when running in Hadoop Local Mode.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to