You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2007/08/31 00:05:28 UTC

svn commit: r571330 - in /lucene/hadoop/trunk: CHANGES.txt src/java/org/apache/hadoop/mapred/Task.java src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java

Author: cutting
Date: Thu Aug 30 15:05:28 2007
New Revision: 571330

URL: http://svn.apache.org/viewvc?rev=571330&view=rev
Log:
HADOOP-1795.  Fix so that jobs can generate output file names with special characters.  Contributed by Frédéric Bertin.

Added:
    lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=571330&r1=571329&r2=571330&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Thu Aug 30 15:05:28 2007
@@ -76,6 +76,9 @@
     IllegalArgumentException in MapWritable.
     (Jim Kellerman via cutting)
 
+    HADOOP-1795.  Fix so that jobs can generate output file names with
+    special characters.  (Frédéric Bertin via cutting)
+
   IMPROVEMENTS
 
     HADOOP-1779. Replace INodeDirectory.getINode() by a getExistingPathINodes()

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java?rev=571330&r1=571329&r2=571330&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java Thu Aug 30 15:05:28 2007
@@ -438,7 +438,7 @@
   
   private Path getFinalPath(Path jobOutputDir, Path taskOutput) {
     URI relativePath = taskOutputPath.toUri().relativize(taskOutput.toUri());
-    return new Path(jobOutputDir, relativePath.toString());
+    return new Path(jobOutputDir, relativePath.getPath());
   }
   
   private void moveTaskOutputs(FileSystem fs, Path jobOutputDir, Path taskOutput) 

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java?rev=571330&view=auto
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSpecialCharactersInOutputPath.java Thu Aug 30 15:05:28 2007
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapred;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * A JUnit test to test that jobs' output filenames are not HTML-encoded (cf HADOOP-1795).
+ */
+public class TestSpecialCharactersInOutputPath extends TestCase {
+  private static final Log LOG =
+    LogFactory.getLog(TestSpecialCharactersInOutputPath.class.getName());
+  
+  private static final String OUTPUT_FILENAME = "result[0]";
+  
+  public static boolean launchJob(String fileSys,
+                                       String jobTracker,
+                                       JobConf conf,
+                                       int numMaps,
+                                       int numReduces) throws IOException {
+    
+    final Path inDir = new Path("/testing/input");
+    final Path outDir = new Path("/testing/output");
+    FileSystem fs = FileSystem.getNamed(fileSys, conf);
+    fs.delete(outDir);
+    if (!fs.mkdirs(inDir)) {
+      LOG.warn("Can't create " + inDir);
+      return false;
+    }
+    // generate an input file
+    DataOutputStream file = fs.create(new Path(inDir, "part-0"));
+    file.writeBytes("foo foo2 foo3");
+    file.close();
+
+    // use WordCount example
+    conf.set("fs.default.name", fileSys);
+    conf.set("mapred.job.tracker", jobTracker);
+    conf.setJobName("foo");
+
+    conf.setInputFormat(TextInputFormat.class);
+    conf.setOutputFormat(SpecialTextOutputFormat.class);
+    conf.setOutputKeyClass(LongWritable.class);
+    conf.setOutputValueClass(Text.class);
+    conf.setMapperClass(IdentityMapper.class);        
+    conf.setReducerClass(IdentityReducer.class);
+    conf.setInputPath(inDir);
+    conf.setOutputPath(outDir);
+    conf.setNumMapTasks(numMaps);
+    conf.setNumReduceTasks(numReduces);
+      
+    // run job and wait for completion
+    RunningJob runningJob = JobClient.runJob(conf);
+      
+    try {
+      assertTrue(runningJob.isComplete());
+      assertTrue(runningJob.isSuccessful());
+      assertTrue("Output folder not found!", fs.exists(new Path("/testing/output/" + OUTPUT_FILENAME)));
+    } catch (NullPointerException npe) {
+      // This NPE should no more happens
+      fail("A NPE should not have happened.");
+    }
+          
+    // return job result
+    LOG.info("job is complete: " + runningJob.isSuccessful());
+    return (runningJob.isSuccessful());
+  }
+  
+  public void testJobWithDFS() throws IOException {
+    String namenode = null;
+    MiniDFSCluster dfs = null;
+    MiniMRCluster mr = null;
+    FileSystem fileSys = null;
+    try {
+      final int taskTrackers = 4;
+      final int jobTrackerPort = 60050;
+      Configuration conf = new Configuration();
+      dfs = new MiniDFSCluster(conf, 1, true, null);
+      fileSys = dfs.getFileSystem();
+      namenode = fileSys.getName();
+      mr = new MiniMRCluster(taskTrackers, namenode, 2);
+      final String jobTrackerName = "localhost:" + mr.getJobTrackerPort();
+      JobConf jobConf = new JobConf();
+      boolean result;
+      result = launchJob(namenode, jobTrackerName, jobConf, 
+                              3, 1);
+      assertTrue(result);
+          
+    } finally {
+      if (fileSys != null) { fileSys.close(); }
+      if (dfs != null) { dfs.shutdown(); }
+      if (mr != null) { mr.shutdown(); }
+    }
+  }
+
+  /** generates output filenames with special characters */
+  static class SpecialTextOutputFormat extends TextOutputFormat {
+    @Override
+    public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name, Progressable progress) throws IOException {
+        return super.getRecordWriter(ignored, job, OUTPUT_FILENAME, progress);
+    }
+  }
+}