From hadoop-user-return-2782-apmail-lucene-hadoop-user-archive=lucene.apache.org@lucene.apache.org Thu Oct 25 17:40:24 2007 Return-Path: Delivered-To: apmail-lucene-hadoop-user-archive@locus.apache.org Received: (qmail 51213 invoked from network); 25 Oct 2007 17:40:23 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 25 Oct 2007 17:40:23 -0000 Received: (qmail 78518 invoked by uid 500); 25 Oct 2007 17:40:04 -0000 Delivered-To: apmail-lucene-hadoop-user-archive@lucene.apache.org Received: (qmail 78508 invoked by uid 500); 25 Oct 2007 17:40:04 -0000 Mailing-List: contact hadoop-user-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-user@lucene.apache.org Delivered-To: mailing list hadoop-user@lucene.apache.org Received: (qmail 78499 invoked by uid 99); 25 Oct 2007 17:40:04 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 25 Oct 2007 10:40:04 -0700 X-ASF-Spam-Status: No, hits=1.2 required=10.0 tests=SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (athena.apache.org: local policy) Received: from [216.145.54.172] (HELO mrout2.yahoo.com) (216.145.54.172) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 25 Oct 2007 17:40:04 +0000 Received: from reasonpublic-lx.corp.yahoo.com (reasonpublic-lx.corp.yahoo.com [10.72.104.164]) by mrout2.yahoo.com (8.13.6/8.13.6/y.out) with ESMTP id l9PHcF7x062604; Thu, 25 Oct 2007 10:38:15 -0700 (PDT) DomainKey-Signature: a=rsa-sha1; s=serpent; d=yahoo-inc.com; c=nofws; q=dns; h=from:to:subject:date:user-agent:cc:references:in-reply-to: mime-version:content-type:content-transfer-encoding: content-disposition:message-id; b=Wr/onJrCFvcVIed1kxOUiKVtcX6r0Z9KrKqadv2YwXzplzjAG3tIl3N0OPOBqKVx From: Benjamin Reed To: hadoop-user@lucene.apache.org Subject: Re: InputFiles, Splits, Maps, Tasks Questions 1.3 Base Date: Thu, 25 Oct 2007 10:38:23 -0700 User-Agent: KMail/1.9.6 (enterprise 0.20070907.709405) Cc: Lance Amundsen References: In-Reply-To: MIME-Version: 1.0 Content-Type: text/plain; charset="iso-8859-1" Content-Transfer-Encoding: 7bit Content-Disposition: inline Message-Id: <200710251038.23654.breed@yahoo-inc.com> X-Virus-Checked: Checked by ClamAV on apache.org I did a patch last year that got similar improvements but still using an external process. (I really like the idea of keeping user code out of the JobTracker and the TaskTracker. It makes things more stable.) See HADOOP-249. It reuses the JVM for a task, which avoids the JVM restart hit. This hit is really bad for cases such as yours. It also avoids the performance hit of doing socket I/O for progress and task info, and instead uses the process pip, which also gives a big performance improvement. Unfortunately, it was never incorporated and now the patch no longer applies. It's really not a big change, but the Hadoop code path to spawn the JVM is a bit convoluted, which made it hard to do the change and makes it hard to bring the patch up-to-date. ben On Thursday 25 October 2007 10:19:59 Lance Amundsen wrote: > So I managed to get my fast InputFormat working.... it does still use the > FS, but in such a way that it improves mapper startup by over 2X. And last > night I got a prototype working that allows the map task to run under the > JVM of the TaskTracker, rather than spawing a new JVM. > > The initial performance look really, really good. I just ran a 1000 map > single input record job, (mappers doing no work however), in a one master, > one slave setup... on my laptop.... It completed in a couple thousand > seconds, or a couple seconds per map. Earlier I did a smaller 100 map job > with a stable quieced system and it came in at about 130 seconds. > > So this prototype can start and end map jobs in 1-2 seconds, and should > scale flatly with respect to nodes in the setup. > > > > > "Owen O'Malley" > m> To > hadoop-user@lucene.apache.org > 10/24/2007 01:05 cc > PM > Subject > Re: InputFiles, Splits, Maps, Tasks > Please respond to Questions 1.3 Base > hadoop-user@lucen > e.apache.org > > On Oct 24, 2007, at 12:42 PM, Doug Cutting wrote: > > Lance Amundsen wrote: > >> OK, that is encouraging. I'll take another pass at it. I succeeded > >> yesterday with an in-memory only InputFormat, but only after I > >> commented > >> out some of the split referencing code, like the following in > >> MapTask.java > >> if (instantiatedSplit instanceof FileSplit) { > >> FileSplit fileSplit = (FileSplit) instantiatedSplit; > >> job.set("map.input.file", fileSplit.getPath().toString()); > >> job.setLong("map.input.start", fileSplit.getStart()); > >> job.setLong("map.input.length", fileSplit.getLength()); > >> } > > > > Yes, that code should not exist, but it shouldn't affect you > > either. You should be subclassing InputSplit, not FileSplit, so > > this code shouldn't operate on your splits. > > That code doesn't do anything if they are non file-splits, so it > absolutely shouldn't break anything. Applications depend on those > attributes to know which split they are working on and there isn't a > better fix until we move to context objects. I know that non- > filesplits work because there are units tests to make sure they don't > break anything. > > -- Owen