Uploaded image for project: 'Hadoop Map/Reduce'
  1. Hadoop Map/Reduce
  2. MAPREDUCE-86

Custom FileSystem class not found during child process initialization

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Not A Problem
    • None
    • None
    • None
    • None

    Description

      If a custom FileSystem class is used for Reducer output, initialization of the child task fails with an uncaught ClassNotFoundException. Trace follows.

      java.lang.RuntimeException: java.lang.ClassNotFoundException: cascading.tap.hadoop.S3HttpFileSystem
      java.io.IOException: java.lang.RuntimeException: java.lang.ClassNotFoundException: cascading.tap.hadoop.S3HttpFileSystem
      at org.apache.hadoop.conf.Configuration.getClass(Configuration.java:607)
      at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:161)
      at org.apache.hadoop.fs.Path.getFileSystem(Path.java:175)
      at org.apache.hadoop.mapred.Task.getTaskOutputPath(Task.java:195)
      at org.apache.hadoop.mapred.Task.setConf(Task.java:400)
      at org.apache.hadoop.mapred.TaskInProgress.getTaskToRun(TaskInProgress.java:733)
      at org.apache.hadoop.mapred.JobInProgress.obtainNewMapTask(JobInProgress.java:568)
      at org.apache.hadoop.mapred.JobTracker.getNewTaskForTaskTracker(JobTracker.java:1409)
      at org.apache.hadoop.mapred.JobTracker.heartbeat(JobTracker.java:1191)
      at sun.reflect.GeneratedMethodAccessor2.invoke(Unknown Source)

      Task.getTaskOutputPath only attempts to force the path into a fully qualified reference. It already handles thrown IOE, and should possibly just catch Exception, or atleast explicitly catch the CNFE. On a catch, can continue to return the original Path instance passed.

      Bar.java
        private Path getTaskOutputPath(JobConf conf) {
          Path p = new Path(conf.getOutputPath(), ("_" + taskId));
          try {
            FileSystem fs = p.getFileSystem(conf);
            return p.makeQualified(fs);
          } catch (IOException ie) {  // SHOULD BE BROADENED?
            LOG.warn(StringUtils.stringifyException(ie));
            return p;
          }
        }
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            cwensel Chris Wensel
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: