[ 
http://issues.apache.org/jira/browse/HADOOP-492?page=comments#action_12442212 ] 
            
arkady borkovsky commented on HADOOP-492:
-----------------------------------------


   [[ Old comment, sent by email on Thu, 31 Aug 2006 10:40:32 -0700 ]]

It may be that Metrics API can be used for this purpose.

However Metrics API  is an "API for reporting performance metric  
information", while this proposal is more application oriented.
I'd like to be able to something like this:
-- in main() of MapReduce job
    JobConf conf ;
    GlobalCounters ggg = conf.addGlobalCounter( "TotalWordCount");

-- in map() ,  
     GlobalCounter  totalWords = reporter.getGlobalCounter(  
"TotalWords");
and whenever it processes a word
     totalWords.inc(1);

-- in the end of main(), after the job has completed.

     int totalWords = 0;
     for (int i=0; i < ggg.size(); i++) {
           totalWords += ggg.get(i);
     }

(I've pretended the GlobalCounter s are always int)

Currently, using vanilla MapReduce would require running two jobs --  
one to count the individual words, another to aggregate the counts (or  
to extract the aggregated counts from the output.)

Although not shown in this example, I assume that ggg is updated real  
time, and main() can run a thread to monitor it while the tasks are  
running.
Also it assumes that task failures and speculative execution are  
handled correctly.






> Global counters
> ---------------
>
>                 Key: HADOOP-492
>                 URL: http://issues.apache.org/jira/browse/HADOOP-492
>             Project: Hadoop
>          Issue Type: New Feature
>          Components: mapred
>            Reporter: arkady borkovsky
>
> It would be nice to have map / reduce job keep aggregated counts for 
> arbitrary events occuring in its tasks -- the numer of records processed, the 
> numer of exceptions of a specific type, the number of sentences in passive 
> voice, whatever the jobs finds useful.
> This can be implemented by tasks periodically sending <name, value> pairs to 
> the jobtracker (in some implementations such messages are piggy-backed on the 
> heartbeats), so that the job tracker stores all the latests values from each 
> task and aggregates them on a request.  It should also make the aggregated 
> values available at the job end.  The value for a task would be flushed when 
> the task fails.
> #491 and #490 may be related to this one.

-- 
This message is automatically generated by JIRA.
-
If you think it was sent incorrectly contact one of the administrators: 
http://issues.apache.org/jira/secure/Administrators.jspa
-
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to