Return-Path: Delivered-To: apmail-hadoop-mapreduce-commits-archive@minotaur.apache.org Received: (qmail 91936 invoked from network); 25 Jan 2010 06:37:25 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 25 Jan 2010 06:37:25 -0000 Received: (qmail 52806 invoked by uid 500); 25 Jan 2010 06:37:24 -0000 Delivered-To: apmail-hadoop-mapreduce-commits-archive@hadoop.apache.org Received: (qmail 52746 invoked by uid 500); 25 Jan 2010 06:37:24 -0000 Mailing-List: contact mapreduce-commits-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-commits@hadoop.apache.org Received: (qmail 52736 invoked by uid 99); 25 Jan 2010 06:37:24 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Jan 2010 06:37:24 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Jan 2010 06:37:16 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id C3CD823889BB; Mon, 25 Jan 2010 06:36:56 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r902717 - in /hadoop/mapreduce/trunk: CHANGES.txt src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java Date: Mon, 25 Jan 2010 06:36:56 -0000 To: mapreduce-commits@hadoop.apache.org From: cdouglas@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100125063656.C3CD823889BB@eris.apache.org> Author: cdouglas Date: Mon Jan 25 06:36:56 2010 New Revision: 902717 URL: http://svn.apache.org/viewvc?rev=902717&view=rev Log: MAPREDUCE-1337. Use generics in StreamJob to improve readability of that class. Contributed by Kay Kay Modified: hadoop/mapreduce/trunk/CHANGES.txt hadoop/mapreduce/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java hadoop/mapreduce/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java Modified: hadoop/mapreduce/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/CHANGES.txt?rev=902717&r1=902716&r2=902717&view=diff ============================================================================== --- hadoop/mapreduce/trunk/CHANGES.txt (original) +++ hadoop/mapreduce/trunk/CHANGES.txt Mon Jan 25 06:36:56 2010 @@ -124,6 +124,9 @@ MAPREDUCE-847. Fix Releaseaudit warning count to zero (Giridharan Kesavan) + MAPREDUCE-1337. Use generics in StreamJob to improve readability of that + class. (Kay Kay via cdouglas) + OPTIMIZATIONS MAPREDUCE-270. Fix the tasktracker to optionally send an out-of-band Modified: hadoop/mapreduce/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?rev=902717&r1=902716&r2=902717&view=diff ============================================================================== --- hadoop/mapreduce/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original) +++ hadoop/mapreduce/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Mon Jan 25 06:36:56 2010 @@ -25,9 +25,7 @@ import java.net.URLEncoder; import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; import java.util.List; -import java.util.ListIterator; import java.util.Map; import java.util.regex.Pattern; import java.util.TreeMap; @@ -192,9 +190,8 @@ } msg("addTaskEnvironment=" + addTaskEnvironment_); - Iterator it = packageFiles_.iterator(); - while (it.hasNext()) { - File f = new File((String) it.next()); + for (final String packageFile : packageFiles_) { + File f = new File(packageFile); if (f.isFile()) { shippedCanonFiles_.add(f.getCanonicalPath()); } @@ -263,11 +260,11 @@ inputSpecs_.add(input); } } - output_ = (String) cmdLine.getOptionValue("output"); + output_ = cmdLine.getOptionValue("output"); - mapCmd_ = (String)cmdLine.getOptionValue("mapper"); - comCmd_ = (String)cmdLine.getOptionValue("combiner"); - redCmd_ = (String)cmdLine.getOptionValue("reducer"); + mapCmd_ = cmdLine.getOptionValue("mapper"); + comCmd_ = cmdLine.getOptionValue("combiner"); + redCmd_ = cmdLine.getOptionValue("reducer"); lazyOutput_ = cmdLine.hasOption("lazyOutput"); @@ -286,21 +283,21 @@ validate(packageFiles_); } - String fsName = (String)cmdLine.getOptionValue("dfs"); + String fsName = cmdLine.getOptionValue("dfs"); if (null != fsName){ LOG.warn("-dfs option is deprecated, please use -fs instead."); config_.set("fs.default.name", fsName); } - additionalConfSpec_ = (String)cmdLine.getOptionValue("additionalconfspec"); - inputFormatSpec_ = (String)cmdLine.getOptionValue("inputformat"); - outputFormatSpec_ = (String)cmdLine.getOptionValue("outputformat"); - numReduceTasksSpec_ = (String)cmdLine.getOptionValue("numReduceTasks"); - partitionerSpec_ = (String)cmdLine.getOptionValue("partitioner"); - inReaderSpec_ = (String)cmdLine.getOptionValue("inputreader"); - mapDebugSpec_ = (String)cmdLine.getOptionValue("mapdebug"); - reduceDebugSpec_ = (String)cmdLine.getOptionValue("reducedebug"); - ioSpec_ = (String)cmdLine.getOptionValue("io"); + additionalConfSpec_ = cmdLine.getOptionValue("additionalconfspec"); + inputFormatSpec_ = cmdLine.getOptionValue("inputformat"); + outputFormatSpec_ = cmdLine.getOptionValue("outputformat"); + numReduceTasksSpec_ = cmdLine.getOptionValue("numReduceTasks"); + partitionerSpec_ = cmdLine.getOptionValue("partitioner"); + inReaderSpec_ = cmdLine.getOptionValue("inputreader"); + mapDebugSpec_ = cmdLine.getOptionValue("mapdebug"); + reduceDebugSpec_ = cmdLine.getOptionValue("reducedebug"); + ioSpec_ = cmdLine.getOptionValue("io"); String[] car = cmdLine.getOptionValues("cacheArchive"); if (null != car && car.length > 0){ @@ -590,7 +587,7 @@ /** @return path to the created Jar file or null if no files are necessary. */ protected String packageJobJar() throws IOException { - ArrayList unjarFiles = new ArrayList(); + ArrayList unjarFiles = new ArrayList(); // Runtime code: ship same version of code as self (job submitter code) // usually found in: build/contrib or build/hadoop--dev-streaming.jar @@ -892,15 +889,11 @@ protected void listJobConfProperties() { msg("==== JobConf properties:"); - Iterator it = jobConf_.iterator(); - TreeMap sorted = new TreeMap(); - while(it.hasNext()) { - Map.Entry en = (Map.Entry)it.next(); + TreeMap sorted = new TreeMap(); + for (final Map.Entry en : jobConf_) { sorted.put(en.getKey(), en.getValue()); } - it = sorted.entrySet().iterator(); - while(it.hasNext()) { - Map.Entry en = (Map.Entry)it.next(); + for (final Map.Entry en: sorted.entrySet()) { msg(en.getKey() + "=" + en.getValue()); } msg("===="); @@ -1005,11 +998,11 @@ protected JobClient jc_; // command-line arguments - protected ArrayList inputSpecs_ = new ArrayList(); // - protected TreeSet seenPrimary_ = new TreeSet(); // + protected ArrayList inputSpecs_ = new ArrayList(); + protected TreeSet seenPrimary_ = new TreeSet(); protected boolean hasSimpleInputSpecs_; - protected ArrayList packageFiles_ = new ArrayList(); // - protected ArrayList shippedCanonFiles_ = new ArrayList(); // + protected ArrayList packageFiles_ = new ArrayList(); + protected ArrayList shippedCanonFiles_ = new ArrayList(); //protected TreeMap userJobConfProps_ = new TreeMap(); protected String output_; protected String mapCmd_; Modified: hadoop/mapreduce/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java URL: http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java?rev=902717&r1=902716&r2=902717&view=diff ============================================================================== --- hadoop/mapreduce/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java (original) +++ hadoop/mapreduce/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingKeyValue.java Mon Jan 25 06:36:56 2010 @@ -24,6 +24,7 @@ import java.io.*; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.mapreduce.JobContext; /** * This class tests hadoopStreaming in MapReduce local mode. @@ -70,7 +71,7 @@ "-input", INPUT_FILE.getAbsolutePath(), "-output", OUTPUT_DIR.getAbsolutePath(), "-mapper", "cat", - "-jobconf", "mapreduce.task.files.preserve.failedtasks=true", + "-jobconf", JobContext.PRESERVE_FAILED_TASK_FILES + "=true", "-jobconf", "stream.non.zero.exit.is.failure=true", "-jobconf", "stream.tmpdir="+System.getProperty("test.build.data","/tmp") };