Преглед на файлове

Merge -r 1203410:1203411 from trunk to branch. FIXES: MAPREDUCE-3415

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1203415 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur преди 13 години
родител
ревизия
a24139e043

+ 146 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/JarFinder.java

@@ -0,0 +1,146 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.util;
+
+import com.google.common.base.Preconditions;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.text.MessageFormat;
+import java.util.Enumeration;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+/**
+ * Finds the Jar for a class. If the class is in a directory in the
+ * classpath, it creates a Jar on the fly with the contents of the directory
+ * and returns the path to that Jar. If a Jar is created, it is created in
+ * the system temporary directory.
+ */
+public class JarFinder {
+
+  private static void zipDir(File dir, String relativePath, ZipOutputStream zos)
+    throws IOException {
+    Preconditions.checkNotNull(relativePath, "relativePath");
+    Preconditions.checkNotNull(zos, "zos");
+    zipDir(dir, relativePath, zos, true);
+    zos.close();
+  }
+
+  private static void zipDir(File dir, String relativePath, ZipOutputStream zos,
+                             boolean start) throws IOException {
+    String[] dirList = dir.list();
+    for (String aDirList : dirList) {
+      File f = new File(dir, aDirList);
+      if (!f.isHidden()) {
+        if (f.isDirectory()) {
+          if (!start) {
+            ZipEntry dirEntry = new ZipEntry(relativePath + f.getName() + "/");
+            zos.putNextEntry(dirEntry);
+            zos.closeEntry();
+          }
+          String filePath = f.getPath();
+          File file = new File(filePath);
+          zipDir(file, relativePath + f.getName() + "/", zos, false);
+        }
+        else {
+          ZipEntry anEntry = new ZipEntry(relativePath + f.getName());
+          zos.putNextEntry(anEntry);
+          InputStream is = new FileInputStream(f);
+          byte[] arr = new byte[4096];
+          int read = is.read(arr);
+          while (read > -1) {
+            zos.write(arr, 0, read);
+            read = is.read(arr);
+          }
+          is.close();
+          zos.closeEntry();
+        }
+      }
+    }
+  }
+
+  private static void createJar(File dir, File jarFile) throws IOException {
+    Preconditions.checkNotNull(dir, "dir");
+    Preconditions.checkNotNull(jarFile, "jarFile");
+    File jarDir = jarFile.getParentFile();
+    if (!jarDir.exists()) {
+      if (!jarDir.mkdirs()) {
+        throw new IOException(MessageFormat.format("could not create dir [{0}]",
+                                                   jarDir));
+      }
+    }
+    JarOutputStream zos = new JarOutputStream(new FileOutputStream(jarFile),
+                                              new Manifest());
+    zipDir(dir, "", zos);
+  }
+
+  /**
+   * Returns the full path to the Jar containing the class. It always return a
+   * JAR.
+   *
+   * @param klass class.
+   *
+   * @return path to the Jar containing the class.
+   */
+  public static String getJar(Class klass) {
+    Preconditions.checkNotNull(klass, "klass");
+    ClassLoader loader = klass.getClassLoader();
+    if (loader != null) {
+      String class_file = klass.getName().replaceAll("\\.", "/") + ".class";
+      try {
+        for (Enumeration itr = loader.getResources(class_file);
+             itr.hasMoreElements(); ) {
+          URL url = (URL) itr.nextElement();
+          String path = url.getPath();
+          if (path.startsWith("file:")) {
+            path = path.substring("file:".length());
+          }
+          path = URLDecoder.decode(path, "UTF-8");
+          if ("jar".equals(url.getProtocol())) {
+            path = URLDecoder.decode(path, "UTF-8");
+            return path.replaceAll("!.*$", "");
+          }
+          else if ("file".equals(url.getProtocol())) {
+            String klassName = klass.getName();
+            klassName = klassName.replace(".", "/") + ".class";
+            path = path.substring(0, path.length() - klassName.length());
+            File baseDir = new File(path);
+            File testDir = new File(System.getProperty("test.build.dir", "target/test-dir"));
+            testDir = testDir.getAbsoluteFile();
+            if (!testDir.exists()) {
+              testDir.mkdirs();
+            }
+            File tempJar = File.createTempFile("hadoop-", "", testDir);
+            tempJar = new File(tempJar.getAbsolutePath() + ".jar");
+            createJar(baseDir, tempJar);
+            return tempJar.getAbsolutePath();
+          }
+        }
+      }
+      catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return null;
+  }
+
+}

+ 42 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestJarFinder.java

@@ -0,0 +1,42 @@
+/**
+ * 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.util;
+
+import org.apache.commons.logging.LogFactory;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+
+public class TestJarFinder {
+
+  @Test
+  public void testAppend() throws Exception {
+
+    //picking a class that is for sure in a JAR in the classpath
+    String jar = JarFinder.getJar(LogFactory.class);
+    Assert.assertTrue(new File(jar).exists());
+
+    //picking a class that is for sure in a directory in the classpath
+    //in this case the JAR is created on the fly
+    jar = JarFinder.getJar(TestJarFinder.class);
+    Assert.assertTrue(new File(jar).exists());
+  }
+
+}

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -44,6 +44,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3169. Create a new MiniMRCluster equivalent which only provides 
     client APIs cross MR1 and MR2. (Ahmed via tucu)
 
+    MAPREDUCE-3169. Create a new MiniMRCluster equivalent which only provides 
+    client APIs cross MR1 and MR2. (Ahmed via tucu)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 0 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml

@@ -101,16 +101,9 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <systemPropertyVariables>
-            <yarn.mr.jar>${project.parent.basedir}/hadoop-mapreduce-client-app/target/hadoop-mapreduce-client-app-${project.version}.jar</yarn.mr.jar>
-          </systemPropertyVariables>
           <environmentVariables>
             <JAVA_HOME>${java.home}</JAVA_HOME>
           </environmentVariables>
-          <additionalClasspathElements>
-            <!-- workaround for JobConf#setJarByClass -->
-            <additionalClasspathElement>${project.build.directory}/${project.artifactId}-${project.version}-tests.jar</additionalClasspathElement>
-          </additionalClasspathElements>
         </configuration>
       </plugin>
     </plugins>

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRClientClusterFactory.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
+import org.apache.hadoop.util.JarFinder;
 
 /**
  * A MiniMRCluster factory. In MR2, it provides a wrapper MiniMRClientCluster
@@ -57,7 +58,8 @@ public class MiniMRClientClusterFactory {
     Job job = Job.getInstance(conf);
 
     job.addFileToClassPath(appJar);
-    job.setJarByClass(caller);
+    String callerJar = JarFinder.getJar(caller);
+    job.setJar(callerJar);
 
     MiniMRYarnCluster miniMRYarnCluster = new MiniMRYarnCluster(caller
         .getName(), noOfNMs);

+ 5 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java

@@ -24,13 +24,13 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.LocalContainerLauncher;
 import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
@@ -45,8 +45,7 @@ import org.apache.hadoop.yarn.service.Service;
  */
 public class MiniMRYarnCluster extends MiniYARNCluster {
 
-  public static final String APPJAR = System.getProperty("yarn.mr.jar", JobConf
-      .findContainingJar(LocalContainerLauncher.class));
+  public static final String APPJAR = JarFinder.getJar(LocalContainerLauncher.class);
 
   private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class);
   private JobHistoryServer historyServer;
@@ -55,7 +54,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
   public MiniMRYarnCluster(String testName) {
     this(testName, 1);
   }
-  
+
   public MiniMRYarnCluster(String testName, int noOfNMs) {
     super(testName, noOfNMs);
     //TODO: add the history server
@@ -88,9 +87,9 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
 
     // Set config for JH Server
-    conf.set(JHAdminConfig.MR_HISTORY_ADDRESS, 
+    conf.set(JHAdminConfig.MR_HISTORY_ADDRESS,
         JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
-    
+
     super.init(conf);
   }
 

+ 1 - 4
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml

@@ -82,7 +82,7 @@
             <goals>
               <goal>jar</goal>
             </goals>
-            <!-- strictly speaking, the unit test is really a regression test. It 
+            <!-- strictly speaking, the unit test is really a regression test. It
                  needs the main jar to be available to be able to run. -->
             <phase>test-compile</phase>
           </execution>
@@ -109,9 +109,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <systemPropertyVariables>
-            <yarn.ds.jar>${project.build.directory}/${project.artifactId}-${project.version}.jar</yarn.ds.jar>
-          </systemPropertyVariables>
           <environmentVariables>
             <JAVA_HOME>${java.home}</JAVA_HOME>
           </environmentVariables>

+ 2 - 3
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -37,8 +37,7 @@ public class TestDistributedShell {
   protected static MiniYARNCluster yarnCluster = null;
   protected static Configuration conf = new Configuration();
 
-  protected static String APPMASTER_JAR = System.getProperty("yarn.ds.jar",
-      JobConf.findContainingJar(ApplicationMaster.class));
+  protected static String APPMASTER_JAR = JarFinder.getJar(ApplicationMaster.class);
 
   @BeforeClass
   public static void setup() throws InterruptedException, IOException {