瀏覽代碼

HADOOP-6346. Add support for specifying unpack pattern regex to RunJar.unJar. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@889018 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 15 年之前
父節點
當前提交
a473f37733

+ 3 - 0
CHANGES.txt

@@ -43,6 +43,9 @@ Trunk (unreleased changes)
 
     HADOOP-6400. Log errors getting Unix UGI. (Todd Lipcon via tomwhite)
 
+    HADOOP-6346. Add support for specifying unpack pattern regex to
+    RunJar.unJar. (Todd Lipcon via tomwhite)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 40 - 0
src/java/org/apache/hadoop/conf/Configuration.java

@@ -47,6 +47,7 @@ import java.util.WeakHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -831,6 +832,45 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       : Enum.valueOf(defaultValue.getDeclaringClass(), val);
   }
 
+  /**
+   * Get the value of the <code>name</code> property as a <code>Pattern</code>.
+   * If no such property is specified, or if the specified value is not a valid
+   * <code>Pattern</code>, then <code>DefaultValue</code> is returned.
+   *
+   * @param name property name
+   * @param defaultValue default value
+   * @return property value as a compiled Pattern, or defaultValue
+   */
+  public Pattern getPattern(String name, Pattern defaultValue) {
+    String valString = get(name);
+    if (null == valString || "".equals(valString)) {
+      return defaultValue;
+    }
+    try {
+      return Pattern.compile(valString);
+    } catch (PatternSyntaxException pse) {
+      LOG.warn("Regular expression '" + valString + "' for property '" +
+               name + "' not valid. Using default", pse);
+      return defaultValue;
+    }
+  }
+
+  /**
+   * Set the given property to <code>Pattern</code>.
+   * If the pattern is passed as null, sets the empty pattern which results in
+   * further calls to getPattern(...) returning the default value.
+   *
+   * @param name property name
+   * @param pattern new value
+   */
+  public void setPattern(String name, Pattern pattern) {
+    if (null == pattern) {
+      set(name, null);
+    } else {
+      set(name, pattern.pattern());
+    }
+  }
+
   /**
    * A class that represents a set of positive integer ranges. It parses 
    * strings of the form: "2-3,5,7-" where ranges are separated by comma and 

+ 63 - 36
src/java/org/apache/hadoop/util/RunJar.java

@@ -18,43 +18,66 @@
 
 package org.apache.hadoop.util;
 
-import java.util.jar.*;
-import java.lang.reflect.*;
+import java.lang.reflect.Array;
+import java.lang.reflect.Method;
+import java.lang.reflect.InvocationTargetException;
 import java.net.URL;
 import java.net.URLClassLoader;
-import java.io.*;
-import java.util.*;
-
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.File;
+import java.util.regex.Pattern;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.jar.JarFile;
+import java.util.jar.JarEntry;
+import java.util.jar.Manifest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.io.IOUtils;
 
 /** Run a Hadoop job jar. */
 public class RunJar {
 
-  /** Unpack a jar file into a directory. */
+  /** Pattern that matches any string */
+  public static final Pattern MATCH_ANY = Pattern.compile(".*");
+
+  /**
+   * Unpack a jar file into a directory.
+   *
+   * This version unpacks all files inside the jar regardless of filename.
+   */
   public static void unJar(File jarFile, File toDir) throws IOException {
+    unJar(jarFile, toDir, MATCH_ANY);
+  }
+
+  /**
+   * Unpack matching files from a jar. Entries inside the jar that do
+   * not match the given pattern will be skipped.
+   *
+   * @param jarFile the .jar file to unpack
+   * @param toDir the destination directory into which to unpack the jar
+   * @param unpackRegex the pattern to match jar entries against
+   */
+  public static void unJar(File jarFile, File toDir, Pattern unpackRegex)
+    throws IOException {
     JarFile jar = new JarFile(jarFile);
     try {
-      Enumeration entries = jar.entries();
+      Enumeration<JarEntry> entries = jar.entries();
       while (entries.hasMoreElements()) {
         JarEntry entry = (JarEntry)entries.nextElement();
-        if (!entry.isDirectory()) {
+        if (!entry.isDirectory() &&
+            unpackRegex.matcher(entry.getName()).matches()) {
           InputStream in = jar.getInputStream(entry);
           try {
             File file = new File(toDir, entry.getName());
-            if (!file.getParentFile().mkdirs()) {
-              if (!file.getParentFile().isDirectory()) {
-                throw new IOException("Mkdirs failed to create " + 
-                                      file.getParentFile().toString());
-              }
-            }
+            ensureDirectory(file.getParentFile());
             OutputStream out = new FileOutputStream(file);
             try {
-              byte[] buffer = new byte[8192];
-              int i;
-              while ((i = in.read(buffer)) != -1) {
-                out.write(buffer, 0, i);
-              }
+              IOUtils.copyBytes(in, out, 8192);
             } finally {
               out.close();
             }
@@ -68,6 +91,18 @@ public class RunJar {
     }
   }
 
+  /**
+   * Ensure the existence of a given directory.
+   *
+   * @throws IOException if it cannot be created and does not already exist
+   */
+  private static void ensureDirectory(File dir) throws IOException {
+    if (!dir.mkdirs() && !dir.isDirectory()) {
+      throw new IOException("Mkdirs failed to create " +
+                            dir.toString());
+    }
+  }
+
   /** Run a Hadoop job jar.  If the main class is not in the jar's manifest,
    * then it must be provided on the command line. */
   public static void main(String[] args) throws Throwable {
@@ -107,22 +142,14 @@ public class RunJar {
     mainClassName = mainClassName.replaceAll("/", ".");
 
     File tmpDir = new File(new Configuration().get("hadoop.tmp.dir"));
-    boolean b = tmpDir.mkdirs();
-    if (!b && !tmpDir.isDirectory()) { 
-      System.err.println("Mkdirs failed to create " + tmpDir);
-      System.exit(-1);
-    }
+    ensureDirectory(tmpDir);
+
     final File workDir = File.createTempFile("hadoop-unjar", "", tmpDir);
-    b = workDir.delete();
-    if (!b) {
+    if (!workDir.delete()) {
       System.err.println("Delete failed for " + workDir);
       System.exit(-1);
     }
-    b = workDir.mkdirs();
-    if (!b && !workDir.isDirectory()) {
-      System.err.println("Mkdirs failed to create " + workDir);
-      System.exit(-1);
-    }
+    ensureDirectory(workDir);
 
     Runtime.getRuntime().addShutdownHook(new Thread() {
         public void run() {
@@ -134,15 +161,15 @@ public class RunJar {
       });
 
     unJar(file, workDir);
-    
+
     ArrayList<URL> classPath = new ArrayList<URL>();
-    classPath.add(new File(workDir+"/").toURL());
-    classPath.add(file.toURL());
-    classPath.add(new File(workDir, "classes/").toURL());
+    classPath.add(new File(workDir+"/").toURI().toURL());
+    classPath.add(file.toURI().toURL());
+    classPath.add(new File(workDir, "classes/").toURI().toURL());
     File[] libs = new File(workDir, "lib").listFiles();
     if (libs != null) {
       for (int i = 0; i < libs.length; i++) {
-        classPath.add(libs[i].toURL());
+        classPath.add(libs[i].toURI().toURL());
       }
     }
     

+ 28 - 0
src/test/core/org/apache/hadoop/conf/TestConfiguration.java

@@ -25,6 +25,7 @@ import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Random;
+import java.util.regex.Pattern;
 
 import junit.framework.TestCase;
 
@@ -364,6 +365,33 @@ public class TestConfiguration extends TestCase {
     assertTrue(fail);
   }
 
+  public void testPattern() throws IOException {
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendProperty("test.pattern1", "");
+    appendProperty("test.pattern2", "(");
+    appendProperty("test.pattern3", "a+b");
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    conf.addResource(fileResource);
+
+    Pattern defaultPattern = Pattern.compile("x+");
+    // Return default if missing
+    assertEquals(defaultPattern.pattern(),
+                 conf.getPattern("xxxxx", defaultPattern).pattern());
+    // Return null if empty and default is null
+    assertNull(conf.getPattern("test.pattern1", null));
+    // Return default for empty
+    assertEquals(defaultPattern.pattern(),
+                 conf.getPattern("test.pattern1", defaultPattern).pattern());
+    // Return default for malformed
+    assertEquals(defaultPattern.pattern(),
+                 conf.getPattern("test.pattern2", defaultPattern).pattern());
+    // Works for correct patterns
+    assertEquals("a+b",
+                 conf.getPattern("test.pattern3", defaultPattern).pattern());
+  }
+
   public void testReload() throws IOException {
     out=new BufferedWriter(new FileWriter(CONFIG));
     startConfig();

+ 109 - 0
src/test/core/org/apache/hadoop/util/TestRunJar.java

@@ -0,0 +1,109 @@
+/**
+ * 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 junit.framework.TestCase;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.jar.JarOutputStream;
+import java.util.regex.Pattern;
+import java.util.zip.ZipEntry;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.apache.hadoop.fs.FileUtil;
+
+public class TestRunJar extends TestCase {
+  private File TEST_ROOT_DIR;
+
+  private static final String TEST_JAR_NAME="test-runjar.jar";
+
+  @Before
+  protected void setUp()
+      throws Exception {
+    TEST_ROOT_DIR =
+        new File(System.getProperty("test.build.data", "/tmp"), getClass()
+            .getSimpleName());
+    if (!TEST_ROOT_DIR.exists()) {
+      TEST_ROOT_DIR.mkdirs();
+    }
+
+    makeTestJar();
+  }
+
+  @After
+  protected void tearDown()
+      throws Exception {
+    FileUtil.fullyDelete(TEST_ROOT_DIR);
+  }
+
+  /**
+   * Construct a jar with two files in it in our
+   * test dir.
+   */
+  private void makeTestJar() throws IOException {
+    File jarFile = new File(TEST_ROOT_DIR, TEST_JAR_NAME);
+    JarOutputStream jstream =
+        new JarOutputStream(new FileOutputStream(jarFile));
+    jstream.putNextEntry(new ZipEntry("foobar.txt"));
+    jstream.closeEntry();
+    jstream.putNextEntry(new ZipEntry("foobaz.txt"));
+    jstream.closeEntry();
+    jstream.close();
+  }
+
+  /**
+   * Test default unjarring behavior - unpack everything
+   */
+  @Test
+  public void testUnJar() throws Exception {
+    File unjarDir = new File(TEST_ROOT_DIR, "unjar-all");
+    assertFalse("unjar dir shouldn't exist at test start",
+                new File(unjarDir, "foobar.txt").exists());
+
+    // Unjar everything
+    RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
+                 unjarDir);
+    assertTrue("foobar unpacked",
+               new File(unjarDir, "foobar.txt").exists());
+    assertTrue("foobaz unpacked",
+               new File(unjarDir, "foobaz.txt").exists());
+
+  }
+
+  /**
+   * Test unjarring a specific regex
+   */
+  public void testUnJarWithPattern() throws Exception {
+    File unjarDir = new File(TEST_ROOT_DIR, "unjar-pattern");
+    assertFalse("unjar dir shouldn't exist at test start",
+                new File(unjarDir, "foobar.txt").exists());
+
+    // Unjar only a regex
+    RunJar.unJar(new File(TEST_ROOT_DIR, TEST_JAR_NAME),
+                 unjarDir,
+                 Pattern.compile(".*baz.*"));
+    assertFalse("foobar not unpacked",
+                new File(unjarDir, "foobar.txt").exists());
+    assertTrue("foobaz unpacked",
+               new File(unjarDir, "foobaz.txt").exists());
+
+  }
+}