Return-Path: Delivered-To: apmail-hadoop-core-user-archive@www.apache.org Received: (qmail 42440 invoked from network); 15 Sep 2008 16:44:05 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 15 Sep 2008 16:44:05 -0000 Received: (qmail 60716 invoked by uid 500); 15 Sep 2008 16:43:55 -0000 Delivered-To: apmail-hadoop-core-user-archive@hadoop.apache.org Received: (qmail 60688 invoked by uid 500); 15 Sep 2008 16:43:55 -0000 Mailing-List: contact core-user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: core-user@hadoop.apache.org Delivered-To: mailing list core-user@hadoop.apache.org Received: (qmail 60662 invoked by uid 99); 15 Sep 2008 16:43:55 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 15 Sep 2008 09:43:55 -0700 X-ASF-Spam-Status: No, hits=4.0 required=10.0 tests=HTML_MESSAGE,RCVD_IN_BL_SPAMCOP_NET,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of owen.omalley@gmail.com designates 209.85.198.224 as permitted sender) Received: from [209.85.198.224] (HELO rv-out-0506.google.com) (209.85.198.224) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 15 Sep 2008 16:42:56 +0000 Received: by rv-out-0506.google.com with SMTP id k40so2033910rvb.29 for ; Mon, 15 Sep 2008 09:43:28 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:received:received:message-id:from:to :in-reply-to:content-type:mime-version:subject:date:references :x-mailer:sender; bh=J8Gg8bluEGitSHUl3d3Y4yqTQ7xrcRXNzQH+7fwE7qo=; b=MCquxZ/h+uJUBJlX4/J3jLmsg8GOf32P0a5yeD4X9MARkFUedoBeewP3a0Vp5sb/44 esUjN5GcmA1g2nhobWK+ZX/BbUmFlugkJk4eT7ki9qjwZ3kXFWvnByp6T4ZEOzOcq2Fj cqT1uK2Y7MhvbXSWXnL7Wir4jqoHdpQ98pd/k= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=message-id:from:to:in-reply-to:content-type:mime-version:subject :date:references:x-mailer:sender; b=AbMDZ1GPas0HP8agGsDEw+gedyVP5/GI6U5aQAVL9lHjK+Trsnl4ZokQm7RQgqhh6z UPSK+xLpBlMaswWpLqASmvTq7lPZtZBUmt+7/00c8v0graFOXjdeb+fVrVnDWpTmq1NU VymIWVd9WyF/UZ3jIXvUqVNrWAHAgIkQX0pGk= Received: by 10.141.74.17 with SMTP id b17mr4927721rvl.119.1221497008248; Mon, 15 Sep 2008 09:43:28 -0700 (PDT) Received: from battlerock-lm.corp.yahoo.com ( [209.131.62.113]) by mx.google.com with ESMTPS id k2sm23043533rvb.1.2008.09.15.09.43.27 (version=TLSv1/SSLv3 cipher=RC4-MD5); Mon, 15 Sep 2008 09:43:27 -0700 (PDT) Message-Id: <88C928C9-0F4E-4688-BB6E-A3005A53C048@apache.org> From: Owen O'Malley To: core-user@hadoop.apache.org In-Reply-To: <4a41ceba0809150613m3da35b6fk6c7999db0ecf4d14@mail.gmail.com> Content-Type: multipart/alternative; boundary=Apple-Mail-2-524154080 Mime-Version: 1.0 (Apple Message framework v926) Subject: Re: Implementing own InputFormat and RecordReader Date: Mon, 15 Sep 2008 09:43:26 -0700 References: <4a41ceba0809150613m3da35b6fk6c7999db0ecf4d14@mail.gmail.com> X-Mailer: Apple Mail (2.926) Sender: Owen O'Malley X-Virus-Checked: Checked by ClamAV on apache.org --Apple-Mail-2-524154080 Content-Type: text/plain; charset=ISO-8859-1; format=flowed; delsp=yes Content-Transfer-Encoding: quoted-printable On Sep 15, 2008, at 6:13 AM, Juho M=E4kinen wrote: > 1) The FileInputFormat.getSplits() returns InputSplit[] array. If my > input file is 128MB and my HDFS block size is 64MB, will it return one > InputSplit or two InputSplits? Your InputFormat needs to define: protected boolean isSplitable(FileSystem fs, Path filename) { return false; } which tells the FileInputFormat.getSplits to not split files. You will =20= end up with a single split for each file. > 2) If my file is splitted into two or more filesystem blocks, how will > hadoop handle the reading of those blocks? As the file must be read in > sequence, will hadoop first copy every block to a machine (if the > blocks aren't already in there) and then start the mapper in this > machine? Do I need to handle the reading and opening multiple blocks, > or will hadoop provide me a simple stream interface which I can use to > read the entire file without worrying if the file is larger than the > HDFS block size? HDFS transparently handles the data motion for you. You can just use =20 FileSystem.open(path) and HDFS will pull the file from the closest =20 location. It doesn't actually move the block to your local disk, just =20= gives it to the application. Basically, you don't need to worry about =20= it. There are two downsides to unsplitable files. The first is that if =20 they are large, the map times can be very long. The second is that the =20= map/reduce scheduler tries to place the tasks close to the data, which =20= it can't do very well if the data spans blocks. Of course if data =20 isn't splitable, you don't have a choice. -- Owen= --Apple-Mail-2-524154080--