You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Doug Cutting (JIRA)" <ji...@apache.org> on 2006/09/07 00:29:23 UTC

[jira] Commented: (HADOOP-509) initTasks method in the JobInProgress class does not correctly configure the classloader with the custom map/reduce jar

    [ http://issues.apache.org/jira/browse/HADOOP-509?page=comments#action_12432985 ] 
            
Doug Cutting commented on HADOOP-509:
-------------------------------------

Yes, the JobTracker should unpack the jar and construct a classpath similar to what is done in TaskRunner for child processes.  Ideally these would both use the same code to build the classpath.  A caching mechanism, like that in HADOOP-452, could be used to minimize DFS access.

> initTasks method in the JobInProgress class does not correctly configure the classloader with the custom map/reduce jar
> -----------------------------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-509
>                 URL: http://issues.apache.org/jira/browse/HADOOP-509
>             Project: Hadoop
>          Issue Type: Bug
>          Components: mapred
>    Affects Versions: 0.5.0
>            Reporter: Thomas Friol
>
> In this part of code :
>         JobConf jd = new JobConf(localJobFile);
>         FileSystem fs = FileSystem.get(conf);
>         String ifClassName = jd.get("mapred.input.format.class");
>         InputFormat inputFormat;
>         if (ifClassName != null && localJarFile != null) {
>           try {
>             ClassLoader loader =
>               new URLClassLoader(new URL[]{ localFs.pathToFile(localJarFile).toURL() });
>             Class inputFormatClass = Class.forName(ifClassName, true, loader);
>             inputFormat = (InputFormat)inputFormatClass.newInstance();
>           } catch (Exception e) {
>             throw new IOException(e.toString());
>           }
>         } else {
>           inputFormat = jd.getInputFormat();
>         }
> I think that the URLClassLoader should be added the URLs to all the jars contained in the lib directory of the mapred jar. Or maybe, the jar should be unjarred and the classpath should be updated as well as it is done into the TaskRunner class :
>       String jar = conf.getJar();
>       if (jar != null) {                      // if jar exists, it into workDir
>         RunJar.unJar(new File(jar), workDir);
>         File[] libs = new File(workDir, "lib").listFiles();
>         if (libs != null) {
>           for (int i = 0; i < libs.length; i++) {
>             classPath.append(sep);            // add libs from jar to classpath
>             classPath.append(libs[i]);
>           }
>         }
>         classPath.append(sep);
>         classPath.append(new File(workDir, "classes"));
>         classPath.append(sep);
>         classPath.append(workDir);
>       }
> Actually, it is not possible to load a custom InputFormat class. Any thoughts ?
> PS : This issue comes following a discussion on the issue https://issues.apache.org/jira/browse/HADOOP-366

-- 
This message is automatically generated by JIRA.
-
If you think it was sent incorrectly contact one of the administrators: http://issues.apache.org/jira/secure/Administrators.jspa
-
For more information on JIRA, see: http://www.atlassian.com/software/jira