Return-Path: Delivered-To: apmail-hadoop-mapreduce-dev-archive@minotaur.apache.org Received: (qmail 39238 invoked from network); 4 Feb 2011 06:14:38 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 4 Feb 2011 06:14:38 -0000 Received: (qmail 80525 invoked by uid 500); 4 Feb 2011 06:14:38 -0000 Delivered-To: apmail-hadoop-mapreduce-dev-archive@hadoop.apache.org Received: (qmail 80180 invoked by uid 500); 4 Feb 2011 06:14:34 -0000 Mailing-List: contact mapreduce-dev-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: mapreduce-dev@hadoop.apache.org Delivered-To: mailing list mapreduce-dev@hadoop.apache.org Received: (qmail 80172 invoked by uid 99); 4 Feb 2011 06:14:33 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Feb 2011 06:14:33 +0000 X-ASF-Spam-Status: No, hits=0.7 required=5.0 tests=RCVD_IN_DNSWL_NONE,SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (nike.apache.org: local policy) Received: from [69.147.107.20] (HELO mrout1-b.corp.re1.yahoo.com) (69.147.107.20) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Feb 2011 06:14:27 +0000 Received: from [10.66.93.133] (evenmonth-dr.eglbp.corp.yahoo.com [10.66.93.133]) by mrout1-b.corp.re1.yahoo.com (8.14.4/8.14.4/y.out) with ESMTP id p146DlMJ052475 for ; Thu, 3 Feb 2011 22:13:48 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=simple/simple; d=yahoo-inc.com; s=cobra; t=1296800029; bh=qy+YhTy329x3DtwKxcXyL+kQ3lT92Sp0+nyYxwg9ZZo=; h=Message-ID:Date:From:MIME-Version:To:Subject:References: In-Reply-To:Content-Type:Content-Transfer-Encoding; b=vicwx6qWljJbz6THeQBY64SowBkit/Fc87kJeztiYQ461uQb43/hw8F3Wbs7yFnDS 91Jaw09d+EmjKHeRBrJw00l0s5zaxBERmya/S1D4xg0Y7zyTN66y43aQ6Ob+We94Xk TpJKf9+RqCxyVQI98F3VoYdEtN99WnCaXmNX9jLU= Message-ID: <4D4B9AED.9060107@yahoo-inc.com> Date: Fri, 04 Feb 2011 11:51:33 +0530 From: Ravi Gummadi User-Agent: Thunderbird 1.5.0.12 (X11/20080411) MIME-Version: 1.0 To: "mapreduce-dev@hadoop.apache.org" Subject: Re: "Map input bytes" vs HDFS_BYTES_READ References: <4D4B7D1A.4030003@yahoo-inc.com> In-Reply-To: Content-Type: text/plain; charset=ISO-8859-1; format=flowed Content-Transfer-Encoding: 7bit X-Virus-Checked: Checked by ClamAV on apache.org Ted Yu wrote: > Ravi: > Can you illustrate the situation where map output doesn't fit in io.sort.mb > ? > > Thanks > Basically, map output is spilled to local disk in chunks of size io.sort.mb( i.e. the value of this config property io.sort.mb set for that job). So if user's map() method in a map task outputs 500MB of data(say reading X MB of input split) and io.sort.mb=128, then first 128MB out of this 500MB becomes 1st spill to disk, 2nd 128MB out of this 500MB becomes 1st spill to disk, etc. Anyway, all these spilled files are merged together to form the final single-map-output-file for that map task. This merging can happen in multiple merges --- for example, if io.sort.factor=2, then only 2 spills are merged at a time, thus resulting in multiple merges with intermediate writes to local disk. So based on the number of spills and intermediate merges happen, the local-file-bytes-read counter can be a lot bigger value compared to the actual map input bytes. Spilled records counter(number of records spilled to disk in the map task) may help for your calculation to some extent. But it is records-count and not bytes. -Ravi > On Thu, Feb 3, 2011 at 8:14 PM, Ravi Gummadi wrote: > > >> Ted Yu wrote: >> >> >>> From my limited experiment, I think "Map input bytes" reflects the number >>> of >>> bytes of local data file(s) when LocalJobRunner is used. >>> >>> Correct me if I am wrong. >>> >>> >>> >> This is correct only if there is a single spill (and not multiple spills) >> i.e. all the map output fits in io.sort.mb. >> >> -Ravi >> >> On Tue, Feb 1, 2011 at 7:52 PM, Harsh J wrote: >> >>> >>> >>>> Each task counts independently of its attempt/other tasks, thereby >>>> making the aggregates easier to control. Final counters are aggregated >>>> only from successfully committed tasks. During the job's run, however, >>>> counters are shown aggregated from the most successful attempts of a >>>> task thus far. >>>> >>>> On Wed, Feb 2, 2011 at 9:09 AM, Ted Yu wrote: >>>> >>>> >>>> >>>>> If map task(s) were retried (mapred.map.max.attempts times), how would >>>>> >>>>> >>>>> >>>> these >>>> >>>> >>>> >>>>> two counters be affected ? >>>>> >>>>> Thanks >>>>> >>>>> On Tue, Feb 1, 2011 at 7:31 PM, Harsh J wrote: >>>>> >>>>> >>>>> >>>>> >>>>>> HDFS_BYTES_READ is a FileSystem interface counter. It directly deals >>>>>> with the FS read (lower level). Map input bytes is what the >>>>>> RecordReader has processed in number of bytes for records being read >>>>>> from the input stream. >>>>>> >>>>>> For plain text files, I believe both counters must report about the >>>>>> same value, were entire records being read with no operation performed >>>>>> on each line. But when you throw in a compressed file, you'll notice >>>>>> that the HDFS_BYTES_READ would be far lesser than Map input bytes >>>>>> since the disk read was low, but the total content stored in record >>>>>> terms was still the same as it would be for an uncompressed file. >>>>>> >>>>>> Hope this clears it. >>>>>> >>>>>> On Wed, Feb 2, 2011 at 8:06 AM, Ted Yu wrote: >>>>>> >>>>>> >>>>>> >>>>>>> In hadoop 0.20.2, what's the relationship between "Map input bytes" >>>>>>> >>>>>>> >>>>>>> >>>>>> and >>>>>> >>>> >>>>> HDFS_BYTES_READ ? >>>>> >>>>>>> >>>>>> name="HDFS_BYTES_READ">203446204073 >>>>>>> >>>>>> name="HDFS_BYTES_WRITTEN">23413127561 >>>>>>> 163502600 >>>>>>> 0 >>>>>> >>>> >>>>> 965922136488 >>>>>>> 296754600 >>>>>>> >>>>>>> Thanks >>>>>>> >>>>>>> >>>>>>> >>>>>>> >>>>>> -- >>>>>> Harsh J >>>>>> www.harshj.com >>>>>> >>>>>> >>>>>> >>>>>> >>>> -- >>>> Harsh J >>>> www.harshj.com >>>> >>>> >>>> >>>>