Return-Path: X-Original-To: apmail-hadoop-common-commits-archive@www.apache.org Delivered-To: apmail-hadoop-common-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C5E6C9F20 for ; Fri, 21 Dec 2012 23:13:06 +0000 (UTC) Received: (qmail 76584 invoked by uid 500); 21 Dec 2012 23:13:06 -0000 Delivered-To: apmail-hadoop-common-commits-archive@hadoop.apache.org Received: (qmail 76535 invoked by uid 500); 21 Dec 2012 23:13:06 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: common-dev@hadoop.apache.org Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 76527 invoked by uid 99); 21 Dec 2012 23:13:06 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 21 Dec 2012 23:13:06 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.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; Fri, 21 Dec 2012 23:13:05 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 499E323889B3 for ; Fri, 21 Dec 2012 23:12:45 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1425181 - /hadoop/common/branches/branch-2/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java Date: Fri, 21 Dec 2012 23:12:45 -0000 To: common-commits@hadoop.apache.org From: jlowe@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20121221231245.499E323889B3@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: jlowe Date: Fri Dec 21 23:12:44 2012 New Revision: 1425181 URL: http://svn.apache.org/viewvc?rev=1425181&view=rev Log: svn merge -c 1425177 FIXES: MAPREDUCE-4793. Problem with adding resources when using both -files and -file to hadoop streaming. Contributed by Jason Lowe Modified: hadoop/common/branches/branch-2/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java Modified: hadoop/common/branches/branch-2/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java?rev=1425181&r1=1425180&r2=1425181&view=diff ============================================================================== --- hadoop/common/branches/branch-2/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java (original) +++ hadoop/common/branches/branch-2/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java Fri Dec 21 23:12:44 2012 @@ -306,7 +306,13 @@ public class StreamJob implements Tool { throw new IllegalArgumentException(e); } } - config_.set("tmpfiles", config_.get("tmpfiles", "") + fileList); + String tmpFiles = config_.get("tmpfiles", ""); + if (tmpFiles.isEmpty()) { + tmpFiles = fileList.toString(); + } else { + tmpFiles = tmpFiles + "," + fileList; + } + config_.set("tmpfiles", tmpFiles); validate(packageFiles_); }